You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2020/05/14 11:16:27 UTC

[lucene-solr] branch master updated: LUCENE-9321, LUCENE-9278: Refactor renderJavadoc to allow relative links with multiple Gradle tasks (#1488)

This is an automated email from the ASF dual-hosted git repository.

uschindler pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new 010168c  LUCENE-9321, LUCENE-9278: Refactor renderJavadoc to allow relative links with multiple Gradle tasks (#1488)
010168c is described below

commit 010168c57b35e402da3d8776c03307af0785a3bd
Author: Uwe Schindler <us...@apache.org>
AuthorDate: Thu May 14 13:16:16 2020 +0200

    LUCENE-9321, LUCENE-9278: Refactor renderJavadoc to allow relative links with multiple Gradle tasks (#1488)
    
    This also automatically collects linked projects by its dependencies, so we don't need to maintain all inter-project javadocs links.
    
    Co-authored-by: Dawid Weiss <dw...@apache.org>
---
 build.gradle                              |   2 +-
 gradle/documentation/documentation.gradle |   7 +-
 gradle/render-javadoc.gradle              | 559 ++++++++++++------------------
 3 files changed, 220 insertions(+), 348 deletions(-)

diff --git a/build.gradle b/build.gradle
index 58afa84..0260fe8 100644
--- a/build.gradle
+++ b/build.gradle
@@ -77,7 +77,6 @@ apply from: file('gradle/ant-compat/folder-layout.gradle')
 // (java, tests)
 apply from: file('gradle/defaults.gradle')
 apply from: file('gradle/defaults-java.gradle')
-apply from: file('gradle/render-javadoc.gradle')
 apply from: file('gradle/testing/defaults-tests.gradle')
 apply from: file('gradle/testing/randomization.gradle')
 apply from: file('gradle/testing/fail-on-no-tests.gradle')
@@ -134,3 +133,4 @@ apply from: file('gradle/ant-compat/forbidden-api-rules-in-sync.gradle')
 
 apply from: file('gradle/documentation/documentation.gradle')
 apply from: file('gradle/documentation/changes-to-html.gradle')
+apply from: file('gradle/render-javadoc.gradle')
diff --git a/gradle/documentation/documentation.gradle b/gradle/documentation/documentation.gradle
index 9ba5c5d..d63569d 100644
--- a/gradle/documentation/documentation.gradle
+++ b/gradle/documentation/documentation.gradle
@@ -22,16 +22,17 @@ configure(rootProject) {
 
     dependsOn allprojects.collect { prj ->
       prj.tasks.matching { task -> task.name in [
-        "changesToHtml"
+        "changesToHtml",
+        "renderSiteJavadoc"
         // TODO: "markdownToHtml"
-        // TODO: "gatherJavadocs"
       ]}
     }
   }
 }
 
+// docroot will point to Lucene and Solr relative directory for each sub-project.
 configure(subprojects.findAll { it.path == ':lucene' || it.path == ':solr' }) {
   ext {
-    docroot = "${project.buildDir}/documentation"
+    docroot = file("${buildDir}/documentation")
   }
 }
diff --git a/gradle/render-javadoc.gradle b/gradle/render-javadoc.gradle
index 583b4ca..6ccf989 100644
--- a/gradle/render-javadoc.gradle
+++ b/gradle/render-javadoc.gradle
@@ -18,17 +18,11 @@
 // generate javadocs by calling javadoc tool
 // see https://docs.oracle.com/en/java/javase/11/tools/javadoc.html
 
-import java.util.stream.Collectors;
-
-// utility function to convert project path to document dir
-// e.g.: ':lucene:analysis:common' => 'analysis/common'
-def pathToDocdir = { path -> path.split(':').drop(2).join('/') }
-
-// escapes an option with single quotes or whitespace to be passed in the options.txt file for
-def escapeJavadocOption = { String s -> (s =~ /[ '"]/) ? ("'" + s.replaceAll(/[\\'"]/, /\\$0/) + "'") : s }
-
 allprojects {
   plugins.withType(JavaPlugin) {
+    ext {
+      relativeDocPath = project.path.replaceFirst(/:\w+:/, "").replace(':', '/')
+    }
 
     // We disable the default javadoc task and have our own
     // javadoc rendering task below. The default javadoc task
@@ -39,394 +33,271 @@ allprojects {
       dependsOn "renderJavadoc"
     }
 
-    task renderJavadoc {
+    task renderJavadoc(type: RenderJavadocTask) {
       description "Generates Javadoc API documentation for the main source code. This directly invokes javadoc tool."
       group "documentation"
 
-      ext {
-        linksource = false
-        linkJUnit = false
-        linkLuceneProjects = []
-        linkSorlProjects = []
-      }
-
       dependsOn sourceSets.main.compileClasspath
+      classpath = sourceSets.main.compileClasspath;
+      srcDirSet = sourceSets.main.java;
 
-      inputs.files { sourceSets.main.java.asFileTree }
-      outputs.dir project.javadoc.destinationDir
-
-      def libName = project.path.startsWith(":lucene") ? "Lucene" : "Solr"
-      def title = "${libName} ${project.version} ${project.name} API".toString()
-
-      // absolute urls for "-linkoffline" option
-      def javaSEDocUrl = "https://docs.oracle.com/en/java/javase/11/docs/api/"
-      def junitDocUrl = "https://junit.org/junit4/javadoc/4.12/"
-      def luceneDocUrl = "https://lucene.apache.org/core/${project.version.replace(".", "_")}".toString()
-      def solrDocUrl = "https://lucene.apache.org/solr/${project.version.replace(".", "_")}".toString()
-
-      def javadocCmd = org.gradle.internal.jvm.Jvm.current().getJavadocExecutable()
-      
-      doFirst {
-        def srcDirs = sourceSets.main.java.srcDirs.findAll { dir -> dir.exists() }
-        def optionsFile = file("${getTemporaryDir()}/javadoc-options.txt")
-
-        def opts = []
-        opts << [ '-overview', file("src/java/overview.html") ]
-        opts << [ '-sourcepath', srcDirs.join(File.pathSeparator) ]
-        opts << [ '-subpackages', project.path.startsWith(':lucene') ? 'org.apache.lucene' : 'org.apache.solr' ]
-        opts << [ '-d', project.javadoc.destinationDir ]
-        opts << '-protected'
-        opts << [ '-encoding', 'UTF-8' ]
-        opts << [ '-charset', 'UTF-8' ]
-        opts << [ '-docencoding', 'UTF-8' ]
-        opts << '-noindex'
-        opts << '-author'
-        opts << '-version'
-        if (linksource) {
-          opts << '-linksource'
-        }
-        opts << '-use'
-        opts << [ '-locale', 'en_US' ]
-        opts << [ '-windowtitle', title ]
-        opts << [ '-doctitle', title ]
-        if (!sourceSets.main.compileClasspath.isEmpty()) {
-          opts << [ '-classpath', sourceSets.main.compileClasspath.asPath ]
-        }
-        opts << [ '-bottom', "<i>Copyright &copy; 2000-${buildYear} Apache Software Foundation. All Rights Reserved.</i>" ]
-
-        opts << [ '-tag', 'lucene.experimental:a:WARNING: This API is experimental and might change in incompatible ways in the next release.' ]
-        opts << [ '-tag', 'lucene.internal:a:NOTE: This API is for internal purposes only and might change in incompatible ways in the next release.' ]
-        opts << [ '-tag', "lucene.spi:t:SPI Name (case-insensitive: if the name is 'htmlStrip', 'htmlstrip' can be used when looking up the service)." ]
-
-        // resolve links to JavaSE and JUnit API
-        opts << [ '-linkoffline', javaSEDocUrl, project(':lucene').file('tools/javadoc/java11/') ]
-        if (linkJUnit) {
-          opts << [ '-linkoffline', junitDocUrl, project(':lucene').file('tools/javadoc/junit/') ]
-        }
-        // resolve inter-project links
-        linkLuceneProjects.collect { path ->
-          opts << [ '-linkoffline', "${luceneDocUrl}/${pathToDocdir(path)}", file(project(path).javadoc.destinationDir) ]
-        }
-        linkSorlProjects.collect { path ->
-          opts << [ '-linkoffline', "${solrDocUrl}/${pathToDocdir(path)}", file(project(path).javadoc.destinationDir) ]
-        }
-
-        opts << [ '--release', 11 ]
-        opts << '-Xdoclint:all,-missing'
-
-        // Temporary file that holds all javadoc options for the current task.
-        String optionsStr = opts.stream()
-          .map{ (it instanceof List) ? it.stream().map{it as String}.map(escapeJavadocOption).collect(Collectors.joining(' ')) : escapeJavadocOption(it as String)}
-          .collect(Collectors.joining('\n'));
-        optionsFile.write(optionsStr, 'UTF-8')
-
-        def outputFile = file("${getTemporaryDir()}/javadoc-output.txt")
-        def result
-        outputFile.withOutputStream { output ->
-          result = project.exec {
-            executable javadocCmd
-
-            standardOutput = output
-            errorOutput = output
-
-            args += [ "@${optionsFile}" ]
-
-            // -J flags can't be passed via options file... (an error "javadoc: error - invalid flag: -J-Xmx512m" occurs.)
-            args += [ "-J-Xmx512m" ]
-            // force locale to be "en_US" (fix for: https://bugs.openjdk.java.net/browse/JDK-8222793)
-            args += [ "-J-Duser.language=en", "-J-Duser.country=US" ]
-
-            ignoreExitValue true
-          }
-        }
-
-        if (result.getExitValue() != 0) {
-          // Pipe the output to console. Intentionally skips any encoding conversion 
-          // and pumps raw bytes.
-          System.out.write(outputFile.bytes)
-
-          def cause
-          try {
-            result.rethrowFailure()
-          } catch (ex) {
-            cause = ex
-          }
-          throw new GradleException("Javadoc generation failed for ${project.path},\n  Options file at: ${optionsFile}\n  Command output at: ${outputFile}", cause)
-        }
-
-        // append some special table css, prettify css
-        ant.concat(destfile: "${project.javadoc.destinationDir}/stylesheet.css", append: "true", fixlastline: "true", encoding: "UTF-8") {
-          filelist(dir: project(":lucene").file("tools/javadoc"), files: "table_padding.css")
-          filelist(dir: project(":lucene").file("tools/prettify"), files: "prettify.css")
-        }
-        // append prettify to scripts
-        ant.concat(destfile: "${project.javadoc.destinationDir}/script.js", append: "true", fixlastline: "true", encoding: "UTF-8") {
-          filelist(dir: project(':lucene').file("tools/prettify"), files: "prettify.js inject-javadocs.js")
-        }
-        ant.fixcrlf(srcdir: project.javadoc.destinationDir, includes: "stylesheet.css script.js", eol: "lf", fixlast: "true", encoding: "UTF-8")
-
-      }
+      outputDir = project.javadoc.destinationDir;
     }
-  }
-}
 
-configure(subprojects.findAll { it.path.startsWith(':lucene') && it.path != ':lucene:core' }) {
+    task renderSiteJavadoc(type: RenderJavadocTask) {
+      description "Generates Javadoc API documentation for the site (relative links)."
+      group "documentation"
 
-  plugins.withType(JavaPlugin) {
-    renderJavadoc {
-      [':lucene:core'].collect { path ->
-        dependsOn "${path}:renderJavadoc"
-        linkLuceneProjects += [ path ]
-      }
+      dependsOn sourceSets.main.compileClasspath
+      classpath = sourceSets.main.compileClasspath;
+      srcDirSet = sourceSets.main.java;
 
-      doLast {
-        // fix for Java 11 Javadoc tool that cannot handle split packages between modules correctly (by removing all the packages which are part of lucene-core)
-        // problem description: [https://issues.apache.org/jira/browse/LUCENE-8738?focusedCommentId=16818106&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16818106]
-        ant.local(name: "element-list-regex") // contains a regex for all package names which are in lucene-core's javadoc
-        ant.loadfile(property: "element-list-regex", srcFile: "${project(':lucene:core').javadoc.destinationDir}/element-list", encoding: "utf-8") {
-          filterchain {
-            tokenfilter(delimoutput: "|") {
-              replacestring(from: ".", to: "\\.")
-            }
-          }
-        }
-        ant.replaceregexp(
-            encoding: "UTF-8",
-            file: "${project.javadoc.destinationDir}/element-list",
-            byline: "true",
-            match: "^(\${element-list-regex})\$",
-            replace: "")
-      }
-    }
-  }
-}
+      relativeProjectLinks = true
 
-configure(subprojects.findAll { it.path.startsWith(':lucene:analysis') && it.path != ':lucene:analysis:common'}) {
-  plugins.withType(JavaPlugin) {
-    renderJavadoc {
-      [':lucene:analysis:common'].collect { path ->
-        dependsOn "${path}:renderJavadoc"
-        linkLuceneProjects += [ path ]
-      }
+      // Place the documentation under Lucene or Solr's documentation directory.
+      // docroot is defined in 'documentation.gradle'
+      outputDir = project.docroot.toPath().resolve(project.relativeDocPath).toFile()
     }
   }
 }
 
-configure(project(':lucene:benchmark')) {
-  plugins.withType(JavaPlugin) {
-    renderJavadoc {
-      [':lucene:memory',
-       ':lucene:highlighter',
-       ':lucene:analysis:common',
-       ':lucene:queryparser',
-       ':lucene:facet',
-       ':lucene:spatial-extras'].collect { path ->
-        dependsOn "${path}:renderJavadoc"
-        linkLuceneProjects += [ path ]
-      }
-    }
+// Set up titles and link up some offline docs for all documentation
+// (they may be unused but this doesn't do any harm).
+def javaJavadocPackages = project.project(':lucene').file('tools/javadoc/java11/')
+def junitJavadocPackages = project.project(':lucene').file('tools/javadoc/junit/')
+allprojects {
+  project.tasks.withType(RenderJavadocTask) {
+    title = "${project.path.startsWith(':lucene') ? 'Lucene' : 'Solr'} ${project.version} ${project.name} API"
+
+    offlineLinks += [
+        "https://docs.oracle.com/en/java/javase/11/docs/api/": javaJavadocPackages,
+        "https://junit.org/junit4/javadoc/4.12/": junitJavadocPackages
+    ]
   }
 }
 
-configure(project(':lucene:classification')) {
-  plugins.withType(JavaPlugin) {
-    renderJavadoc {
-      [':lucene:queries', ':lucene:analysis:common', ':lucene:grouping'].collect { path ->
-        dependsOn "${path}:renderJavadoc"
-        linkLuceneProjects += [ path ]
-      }
+// Fix for Java 11 Javadoc tool that cannot handle split packages between modules correctly.
+// (by removing all the packages which are part of lucene-core)
+// See: https://issues.apache.org/jira/browse/LUCENE-8738?focusedCommentId=16818106&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16818106
+configure(subprojects.findAll { it.path.startsWith(':lucene') && it.path != ':lucene:core' }) {
+  project.tasks.withType(RenderJavadocTask) {
+    doLast {
+      Set luceneCorePackages = file("${project(':lucene:core').tasks[name].outputDir}/element-list").readLines('UTF-8').toSet();
+      File elementFile = file("${outputDir}/element-list");
+      List elements = elementFile.readLines('UTF-8');
+      elements.removeAll(luceneCorePackages)
+      elementFile.write(elements.join('\n').concat('\n'), 'UTF-8');
     }
   }
 }
 
 configure(project(':lucene:demo')) {
-  plugins.withType(JavaPlugin) {
-    renderJavadoc {
-      [':lucene:analysis:common',
-       ':lucene:queryparser',
-       ':lucene:queries',
-       ':lucene:facet',
-       ':lucene:expressions'].collect { path ->
-        dependsOn "${path}:renderJavadoc"
-        linkLuceneProjects += [ path ]
-      }
-
-      // we link the example source in the javadocs, as it's ref'ed elsewhere
-      linksource = true
-    }
+  project.tasks.withType(RenderJavadocTask) {
+    // For the demo, we link the example source in the javadocs, as it's ref'ed elsewhere
+    linksource = true
   }
 }
 
-configure(project(':lucene:grouping')) {
-  plugins.withType(JavaPlugin) {
-    renderJavadoc {
-      [':lucene:queries'].collect { path ->
-        dependsOn "${path}:renderJavadoc"
-        linkLuceneProjects += [ path ]
-      }
-    }
+// Disable Javadoc rendering for these projects.
+configure(subprojects.findAll { it.path in [
+    ':solr:solr-ref-guide',
+    ':solr:server',
+    ':solr:webapp']}) {
+  project.tasks.withType(RenderJavadocTask) {
+    enabled = false
   }
 }
 
-configure(project(':lucene:highlighter')) {
-  plugins.withType(JavaPlugin) {
-    renderJavadoc {
-      [':lucene:memory'].collect { path ->
-        dependsOn "${path}:renderJavadoc"
-        linkLuceneProjects += [ path ]
+// Add cross-project documentation task dependencies:
+// - each RenderJavaDocs task gets a dependency to all tasks with same name in its dependencies
+// - the dependency is using dependsOn with a closure to enable lazy evaluation
+configure(subprojects) {
+  project.tasks.withType(RenderJavadocTask) { task ->
+    task.dependsOn {
+      task.project.configurations.implementation.allDependencies.withType(ProjectDependency).collect { dep ->
+        def otherProject = dep.dependencyProject
+        return otherProject.tasks.findByName(task.name)
       }
     }
   }
 }
 
-configure(project(':lucene:monitor')) {
-  plugins.withType(JavaPlugin) {
-    renderJavadoc {
-      [':lucene:memory', ':lucene:analysis:common', ':lucene:queryparser'].collect { path ->
-        dependsOn "${path}:renderJavadoc"
-        linkLuceneProjects += [ path ]
-      }
-    }
+class RenderJavadocTask extends DefaultTask {
+  @InputFiles
+  @SkipWhenEmpty
+  SourceDirectorySet srcDirSet;
+  
+  @OutputDirectory
+  File outputDir
+
+  @InputFiles
+  @Classpath
+  FileCollection classpath
+
+  @Input
+  String title
+
+  @Input
+  boolean linksource = false
+
+  @Input
+  boolean relativeProjectLinks = false
+
+  @Input
+  def offlineLinks = [:]
+
+  @Input
+  def luceneDocUrl = project.propertyOrDefault('lucene.javadoc.url', "https://lucene.apache.org/core/${project.baseVersion.replace(".", "_")}")
+
+  @Input
+  def solrDocUrl = project.propertyOrDefault('solr.javadoc.url', "https://lucene.apache.org/solr/${project.baseVersion.replace(".", "_")}")
+  
+  /** Utility method to recursively collect all tasks with same name like this one that we depend on */
+  private Set findRenderTasksInDependencies() {
+    Set found = []
+    def collectDeps
+    collectDeps = { task -> task.taskDependencies.getDependencies(task).findAll{ it.name == this.name && it.enabled && !found.contains(it) }.each{
+      found << it
+      collectDeps(it)
+    }}
+    collectDeps(this)
+    return found
   }
-}
 
-configure(project(':lucene:queryparser')) {
-  plugins.withType(JavaPlugin) {
-    renderJavadoc {
-      [':lucene:queries', ':lucene:sandbox'].collect { path ->
-        dependsOn "${path}:renderJavadoc"
-        linkLuceneProjects += [ path ]
-      }
+  @TaskAction
+  public void render() {
+    def javadocCmd = org.gradle.internal.jvm.Jvm.current().getJavadocExecutable()
+
+    def srcDirs = srcDirSet.srcDirs.findAll { dir -> dir.exists() }
+    def optionsFile = project.file("${getTemporaryDir()}/javadoc-options.txt")
+    
+    // create the directory, so relative link calculation knows that it's a directory:
+    outputDir.mkdirs();
+    
+    def opts = []
+    opts << [ '-overview', project.file("${srcDirs[0]}/overview.html") ]
+    opts << [ '-sourcepath', srcDirs.join(File.pathSeparator) ]
+    opts << [ '-subpackages', project.path.startsWith(':lucene') ? 'org.apache.lucene' : 'org.apache.solr' ]
+    opts << [ '-d', outputDir ]
+    opts << '-protected'
+    opts << [ '-encoding', 'UTF-8' ]
+    opts << [ '-charset', 'UTF-8' ]
+    opts << [ '-docencoding', 'UTF-8' ]
+    opts << '-noindex'
+    opts << '-author'
+    opts << '-version'
+    if (linksource) {
+      opts << '-linksource'
     }
-  }
-}
-
-configure(project(':lucene:replicator')) {
-  plugins.withType(JavaPlugin) {
-    renderJavadoc {
-      [':lucene:facet'].collect { path ->
-        dependsOn "${path}:renderJavadoc"
-        linkLuceneProjects += [ path ]
-      }
+    opts << '-use'
+    opts << [ '-locale', 'en_US' ]
+    opts << [ '-windowtitle', title ]
+    opts << [ '-doctitle', title ]
+    if (!classpath.isEmpty()) {
+      opts << [ '-classpath', classpath.asPath ]
     }
-  }
-}
-
-configure(project(':lucene:spatial-extras')) {
-  plugins.withType(JavaPlugin) {
-    renderJavadoc {
-      [':lucene:spatial3d'].collect { path ->
-        dependsOn "${path}:renderJavadoc"
-        linkLuceneProjects += [ path ]
+    opts << [ '-bottom', "<i>Copyright &copy; 2000-${project.buildYear} Apache Software Foundation. All Rights Reserved.</i>" ]
+
+    opts << [ '-tag', 'lucene.experimental:a:WARNING: This API is experimental and might change in incompatible ways in the next release.' ]
+    opts << [ '-tag', 'lucene.internal:a:NOTE: This API is for internal purposes only and might change in incompatible ways in the next release.' ]
+    opts << [ '-tag', "lucene.spi:t:SPI Name (case-insensitive: if the name is 'htmlStrip', 'htmlstrip' can be used when looking up the service)." ]
+
+    def allOfflineLinks = [:]
+    allOfflineLinks.putAll(offlineLinks)
+
+    // Resolve inter-project links:
+    // - find all (enabled) tasks this tasks depends on (with same name), calling findRenderTasksInDependencies()
+    // - sort the tasks preferring those whose project name equals 'core', then lexigraphical by path
+    // - for each task get output dir to create relative or absolute link
+    findRenderTasksInDependencies()
+      .sort(false, Comparator.comparing{ (it.project.name != 'core') as Boolean }.thenComparing(Comparator.comparing{ it.path }))
+      .each{ otherTask ->
+        def otherProject = otherTask.project
+        // For relative links we compute the actual relative link between projects.
+        def crossLuceneSolr = (otherProject.docroot != project.docroot)
+        if (relativeProjectLinks && !crossLuceneSolr) {
+          def pathTo = otherTask.outputDir.toPath().toAbsolutePath()
+          def pathFrom = outputDir.toPath().toAbsolutePath()
+          def relative = pathFrom.relativize(pathTo).toString().replace(File.separator, '/')
+          opts << [ '-link', relative ]
+        } else {
+          // For absolute links, we determine the target URL by assembling the full URL.
+          def base = otherProject.path.startsWith(":lucene") ? luceneDocUrl : solrDocUrl
+          allOfflineLinks.put("${base}/${otherProject.relativeDocPath}/".toString(), otherTask.outputDir)
+        }
       }
-    }
-  }
-}
-
-configure(project(':lucene:suggest')) {
-  plugins.withType(JavaPlugin) {
-    renderJavadoc {
-      [':lucene:analysis:common'].collect { path ->
-        dependsOn "${path}:renderJavadoc"
-        linkLuceneProjects += [ path ]
+    
+    // Add offline links.
+    allOfflineLinks.each { url, dir ->
+      // Some sanity check/ validation here to ensure dir/package-list or dir/element-list is present.
+      if (!project.file("$dir/package-list").exists() &&
+          !project.file("$dir/element-list").exists()) {
+        throw new GradleException("Expected pre-rendered package-list or element-list at ${dir}.")
       }
+      opts << [ '-linkoffline', url, dir ]
     }
-  }
-}
 
-configure(project(':lucene:test-framework')) {
-  plugins.withType(JavaPlugin) {
+    opts << [ '--release', 11 ]
+    opts << '-Xdoclint:all,-missing'
 
-    renderJavadoc {
-      [':lucene:codecs'].collect { path ->
-        dependsOn "${path}:renderJavadoc"
-        linkLuceneProjects += [ path ]
-      }
-      linkJUnit = true
-    }
-  }
-}
+    // Temporary file that holds all javadoc options for the current task.
+    optionsFile.withWriter("UTF-8", { writer ->
+      // escapes an option with single quotes or whitespace to be passed in the options.txt file for
+      def escapeJavadocOption = { String s -> (s =~ /[ '"]/) ? ("'" + s.replaceAll(/[\\'"]/, /\\$0/) + "'") : s }
 
-configure(subprojects.findAll { it.path.startsWith(':solr') }) {
-  plugins.withType(JavaPlugin) {
-    def hasJavdocsTask = project.tasks.collect { it.name }.contains('renderJavadoc')
-    if (hasJavdocsTask) {
-      renderJavadoc {
-        [':lucene:core',
-         ':lucene:analysis:common',
-         ':lucene:analysis:icu',
-         ':lucene:analysis:kuromoji',
-         ':lucene:analysis:nori',
-         ':lucene:analysis:morfologik',
-         ':lucene:analysis:phonetic',
-         ':lucene:analysis:smartcn',
-         ':lucene:analysis:stempel',
-         ':lucene:backward-codecs',
-         ':lucene:codecs',
-         ':lucene:expressions',
-         ':lucene:suggest',
-         ':lucene:grouping',
-         ':lucene:join',
-         ':lucene:queries',
-         ':lucene:queryparser',
-         ':lucene:highlighter',
-         ':lucene:memory',
-         ':lucene:misc',
-         ':lucene:classification',
-         ':lucene:spatial-extras'].collect { path ->
-          dependsOn "${path}:renderJavadoc"
-          linkLuceneProjects += [ path ]
+      opts.each { entry ->
+        if (entry instanceof List) {
+          writer.write(entry.collect { escapeJavadocOption(it as String) }.join(" "))
+        } else {
+          writer.write(escapeJavadocOption(entry as String))
         }
+        writer.write('\n')
       }
-    }
-  }
-}
+    })
 
-configure(project(':solr:core')) {
-  plugins.withType(JavaPlugin) {
-    // specialized to ONLY depend on solrj
-    renderJavadoc {
-      [':solr:solrj'].collect { path ->
-        dependsOn "${path}:renderJavadoc"
-        linkSorlProjects += [ path ]
-      }
-    }
-  }
-}
+    def outputFile = project.file("${getTemporaryDir()}/javadoc-output.txt")
+    def result
+    outputFile.withOutputStream { output ->
+      result = project.exec {
+        executable javadocCmd
 
-configure(subprojects.findAll { it.path.startsWith(':solr:contrib') }) {
-  plugins.withType(JavaPlugin) {
-    renderJavadoc {
-      [':solr:solrj', ':solr:core'].collect { path ->
-        dependsOn "${path}:renderJavadoc"
-        linkSorlProjects += [ path ]
+        standardOutput = output
+        errorOutput = output
+
+        args += [ "@${optionsFile}" ]
+
+        // -J flags can't be passed via options file... (an error "javadoc: error - invalid flag: -J-Xmx512m" occurs.)
+        args += [ "-J-Xmx512m" ]
+        // force locale to be "en_US" (fix for: https://bugs.openjdk.java.net/browse/JDK-8222793)
+        args += [ "-J-Duser.language=en", "-J-Duser.country=US" ]
+
+        ignoreExitValue true
       }
     }
-  }
-}
 
-configure(project(':solr:contrib:dataimporthandler-extras')) {
-  plugins.withType(JavaPlugin) {
-    renderJavadoc {
-      [':solr:contrib:dataimporthandler'].collect { path ->
-        dependsOn "${path}:renderJavadoc"
-        linkSorlProjects += [ path ]
+    if (result.getExitValue() != 0) {
+      // Pipe the output to console. Intentionally skips any encoding conversion 
+      // and pumps raw bytes.
+      System.out.write(outputFile.bytes)
+
+      def cause
+      try {
+        result.rethrowFailure()
+      } catch (ex) {
+        cause = ex
       }
+      throw new GradleException("Javadoc generation failed for ${project.path},\n  Options file at: ${optionsFile}\n  Command output at: ${outputFile}", cause)
     }
-  }
-}
 
-configure(project(':solr:test-framework')) {
-  plugins.withType(JavaPlugin) {
-    renderJavadoc {
-      linkJUnit = true
+    // append some special table css, prettify css
+    ant.concat(destfile: "${outputDir}/stylesheet.css", append: "true", fixlastline: "true", encoding: "UTF-8") {
+      filelist(dir: project.project(":lucene").file("tools/javadoc"), files: "table_padding.css")
+      filelist(dir: project.project(":lucene").file("tools/prettify"), files: "prettify.css")
     }
-  }
-}
-
-
-configure(subprojects.findAll { it.path in [':solr:solr-ref-guide', ':solr:server', ':solr:webapp']}) {
-  afterEvaluate {
-    project.tasks.findByPath("renderJavadoc").enabled = false
+    // append prettify to scripts
+    ant.concat(destfile: "${outputDir}/script.js", append: "true", fixlastline: "true", encoding: "UTF-8") {
+      filelist(dir: project.project(':lucene').file("tools/prettify"), files: "prettify.js inject-javadocs.js")
+    }
+    ant.fixcrlf(srcdir: outputDir, includes: "stylesheet.css script.js", eol: "lf", fixlast: "true", encoding: "UTF-8")
   }
 }