You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2021/10/13 09:51:17 UTC

[lucene] branch main updated: LUCENE-9488: rewrite distribution assembly, signing and checksum generation (#372)

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

dweiss pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/main by this push:
     new 8bcc3dc  LUCENE-9488: rewrite distribution assembly, signing and checksum generation (#372)
8bcc3dc is described below

commit 8bcc3dc430b981dc75c7e005592848eacc745b16
Author: Dawid Weiss <da...@carrotsearch.com>
AuthorDate: Wed Oct 13 11:50:58 2021 +0200

    LUCENE-9488: rewrite distribution assembly, signing and checksum generation (#372)
---
 build.gradle                                       |   6 +-
 .../java/org/apache/lucene/gradle/Checksum.java    | 195 ++++++++++++++++
 gradle/documentation/changes-to-html.gradle        |  12 +
 gradle/globals.gradle                              |   6 +-
 gradle/help.gradle                                 |   2 +-
 gradle/maven/maven-to-local-m2.gradle              |  30 +++
 gradle/maven/maven-to-nexus-releases.gradle        |  79 +++++++
 gradle/maven/maven-to-nexus-snapshots.gradle       |  76 ++++++
 .../publications-maven.gradle}                     | 203 ++++++----------
 gradle/maven/publications.gradle                   |  57 +++++
 gradle/publishing/distribution.gradle              |  45 ----
 gradle/publishing/maven-local.gradle               |  68 ------
 help/gpgSigning.txt                                |  70 ------
 help/publishing.txt                                | 134 +++++++++++
 lucene/distribution/artifact-signing.gradle        |  43 ++++
 lucene/distribution/binary-artifacts.gradle        |  73 ++++++
 lucene/distribution/binary-release.gradle          | 125 ++++++++++
 lucene/distribution/build.gradle                   | 109 +++++++++
 lucene/distribution/collect-maven-artifacts.gradle |  58 +++++
 lucene/distribution/source-release.gradle          |  50 ++++
 lucene/packaging/build.gradle                      | 256 ---------------------
 settings.gradle                                    |   2 +-
 22 files changed, 1117 insertions(+), 582 deletions(-)

diff --git a/build.gradle b/build.gradle
index e10e43a..c2f6351 100644
--- a/build.gradle
+++ b/build.gradle
@@ -120,9 +120,8 @@ apply from: file('gradle/testing/fail-on-no-tests.gradle')
 apply from: file('gradle/testing/alternative-jdk-support.gradle')
 apply from: file('gradle/java/jar-manifest.gradle')
 
-// Publishing and releasing
-apply from: file('gradle/publishing/defaults-maven.gradle')
-apply from: file('gradle/publishing/distribution.gradle')
+// Maven artifact publishing.
+apply from: file('gradle/maven/publications.gradle')
 
 // IDE support, settings and specials.
 apply from: file('gradle/ide/intellij-idea.gradle')
@@ -162,7 +161,6 @@ apply from: file('gradle/datasets/external-datasets.gradle')
 apply from: file('gradle/native/disable-native.gradle')
 
 // Additional development aids.
-apply from: file('gradle/publishing/maven-local.gradle')
 apply from: file('gradle/testing/per-project-summary.gradle')
 apply from: file('gradle/testing/slowest-tests-at-end.gradle')
 apply from: file('gradle/testing/failed-tests-at-end.gradle')
diff --git a/buildSrc/src/main/java/org/apache/lucene/gradle/Checksum.java b/buildSrc/src/main/java/org/apache/lucene/gradle/Checksum.java
new file mode 100644
index 0000000..0dab9dc
--- /dev/null
+++ b/buildSrc/src/main/java/org/apache/lucene/gradle/Checksum.java
@@ -0,0 +1,195 @@
+/*
+ * 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.
+ */
+
+/*
+ * A task that generates SHA512 checksums. Cloned from:
+ * https://github.com/gradle/gradle-checksum/blob/03351de/src/main/java/org/gradle/crypto/checksum/Checksum.java
+ * with custom fixes to make the checksum palatable to shasum, see pending PR:
+ * https://github.com/gradle/gradle-checksum/pull/4
+ *
+ * Original license ASL:
+ * https://github.com/gradle/gradle-checksum/blob/03351de/LICENSE
+ */
+
+package org.apache.lucene.gradle;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.gradle.api.DefaultTask;
+import org.gradle.api.GradleException;
+import org.gradle.api.file.FileCollection;
+import org.gradle.api.file.FileType;
+import org.gradle.api.tasks.Input;
+import org.gradle.api.tasks.InputFiles;
+import org.gradle.api.tasks.OutputDirectory;
+import org.gradle.api.tasks.TaskAction;
+import org.gradle.work.Incremental;
+import org.gradle.work.InputChanges;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.util.Locale;
+
+public class Checksum extends DefaultTask {
+  private FileCollection files;
+  private File outputDir;
+  private Algorithm algorithm;
+
+  public enum Algorithm {
+    MD5(new DigestUtils(DigestUtils.getMd5Digest())),
+    SHA256(new DigestUtils(DigestUtils.getSha256Digest())),
+    SHA384(new DigestUtils(DigestUtils.getSha384Digest())),
+    SHA512(new DigestUtils(DigestUtils.getSha512Digest()));
+
+    private final DigestUtils hashFunction;
+
+    Algorithm(DigestUtils hashFunction) {
+      this.hashFunction = hashFunction;
+    }
+
+    public String getExtension() {
+      return name().toLowerCase(Locale.ROOT);
+    }
+  }
+
+  public Checksum() {
+    outputDir = new File(getProject().getBuildDir(), "checksums");
+    algorithm = Algorithm.SHA256;
+  }
+
+  @InputFiles
+  @Incremental
+  public FileCollection getFiles() {
+    return files;
+  }
+
+  public void setFiles(FileCollection files) {
+    this.files = files;
+  }
+
+  @Input
+  public Algorithm getAlgorithm() {
+    return algorithm;
+  }
+
+  public void setAlgorithm(Algorithm algorithm) {
+    this.algorithm = algorithm;
+  }
+
+  @OutputDirectory
+  public File getOutputDir() {
+    return outputDir;
+  }
+
+  public void setOutputDir(File outputDir) {
+    if (outputDir.exists() && !outputDir.isDirectory()) {
+      throw new IllegalArgumentException("Output directory must be a directory.");
+    }
+    this.outputDir = outputDir;
+  }
+
+  @TaskAction
+  public void generateChecksumFiles(InputChanges inputChanges) throws IOException {
+    if (!getOutputDir().exists()) {
+      if (!getOutputDir().mkdirs()) {
+        throw new IOException("Could not create directory:" + getOutputDir());
+      }
+    }
+
+    if (!inputChanges.isIncremental()) {
+      getProject().delete(allPossibleChecksumFiles());
+    }
+
+    inputChanges
+        .getFileChanges(getFiles())
+        .forEach(
+            fileChange -> {
+              if (fileChange.getFileType() == FileType.DIRECTORY) {
+                getProject()
+                    .getLogger()
+                    .warn("Checksums can't be applied to directories: " + fileChange.getFile());
+                return;
+              }
+
+              File input = fileChange.getFile();
+              switch (fileChange.getChangeType()) {
+                case REMOVED:
+                  if (input.isFile()) {
+                    getProject().delete(outputFileFor(input));
+                  }
+                  break;
+
+                case ADDED:
+                case MODIFIED:
+                  input = fileChange.getFile();
+                  if (input.isFile()) {
+                    File checksumFile = outputFileFor(input);
+
+                    try {
+                      String checksum = algorithm.hashFunction.digestAsHex(input).trim();
+
+                      /*
+                       * https://man7.org/linux/man-pages/man1/sha1sum.1.html
+                       *
+                       * The default mode is to print a line with checksum, a space, a character
+                       * indicating input mode ('*' for binary, ' ' for text or where
+                       * binary is insignificant), and name for each FILE.
+                       */
+                      boolean binaryMode = true;
+
+                      Files.writeString(
+                          checksumFile.toPath(),
+                          String.format(
+                              Locale.ROOT,
+                              "%s %s%s",
+                              checksum,
+                              binaryMode ? "*" : " ",
+                              input.getName()),
+                          StandardCharsets.UTF_8);
+                    } catch (IOException e) {
+                      throw new GradleException("Trouble creating checksum: " + e.getMessage(), e);
+                    }
+                  }
+                  break;
+                default:
+                  throw new RuntimeException();
+              }
+            });
+  }
+
+  private File outputFileFor(File inputFile) {
+    return new File(getOutputDir(), inputFile.getName() + "." + algorithm.getExtension());
+  }
+
+  private FileCollection allPossibleChecksumFiles() {
+    FileCollection possibleFiles = null;
+    for (Algorithm algo : Algorithm.values()) {
+      if (possibleFiles == null) {
+        possibleFiles = filesFor(algo);
+      } else {
+        possibleFiles = possibleFiles.plus(filesFor(algo));
+      }
+    }
+    return possibleFiles;
+  }
+
+  private FileCollection filesFor(final Algorithm algo) {
+    return getProject()
+        .fileTree(getOutputDir(), files -> files.include("**/*." + algo.toString().toLowerCase()));
+  }
+}
diff --git a/gradle/documentation/changes-to-html.gradle b/gradle/documentation/changes-to-html.gradle
index b11a027..2d3f1f6 100644
--- a/gradle/documentation/changes-to-html.gradle
+++ b/gradle/documentation/changes-to-html.gradle
@@ -22,6 +22,18 @@ configure(project(':lucene:documentation')) {
     siteDir = resources
     script = file("${resources}/changes2html.pl")
   }
+
+  // Make the rendered HTML of changes available as a separate
+  // artifact for the distribution.
+  configurations {
+    changesHtml
+  }
+
+  artifacts {
+    changesHtml changesToHtml.targetDir, {
+      builtBy changesToHtml
+    }
+  }
 }
 
 // compile changes.txt into an html file
diff --git a/gradle/globals.gradle b/gradle/globals.gradle
index 4f4ca7d..4650a11 100644
--- a/gradle/globals.gradle
+++ b/gradle/globals.gradle
@@ -47,7 +47,11 @@ allprojects {
 
     // System environment variable or default.
     envOrDefault = { envName, defValue ->
-      return Objects.requireNonNullElse(System.getenv(envName), defValue);
+      def result = System.getenv(envName)
+      if (result == null) {
+        result = defValue
+      }
+      return result
     }
 
     // Either a project, system property, environment variable or default value.
diff --git a/gradle/help.gradle b/gradle/help.gradle
index 2cadf1b..83d6181 100644
--- a/gradle/help.gradle
+++ b/gradle/help.gradle
@@ -29,7 +29,7 @@ configure(rootProject) {
       ["Regeneration", "help/regeneration.txt", "How to refresh generated and derived resources."],
       ["Git", "help/git.txt", "Git assistance and guides."],
       ["IDEs", "help/IDEs.txt", "IDE support."],
-      ["GpgSigning", "help/gpgSigning.txt", "Signing artifacts with GPG."]
+      ["Publishing", "help/publishing.txt", "Maven and other artifact publishing, signing, etc."],
   ]
 
   helpFiles.each { section, path, sectionInfo ->
diff --git a/gradle/maven/maven-to-local-m2.gradle b/gradle/maven/maven-to-local-m2.gradle
new file mode 100644
index 0000000..d63e455
--- /dev/null
+++ b/gradle/maven/maven-to-local-m2.gradle
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+
+// Configure on-demand maven publishing into ~/.m2 for developers' convenience.
+
+configure(rootProject) {
+  task mavenToLocal() {
+    group "Distribution"
+    description "Publish Lucene Maven artifacts to ~/.m2 repository."
+
+    dependsOn rootProject.ext.mavenProjects.collect {
+      it.tasks.matching { it.name == "publishJarsPublicationToMavenLocal" }
+    }
+  }
+}
\ No newline at end of file
diff --git a/gradle/maven/maven-to-nexus-releases.gradle b/gradle/maven/maven-to-nexus-releases.gradle
new file mode 100644
index 0000000..b08ccd1
--- /dev/null
+++ b/gradle/maven/maven-to-nexus-releases.gradle
@@ -0,0 +1,79 @@
+/*
+ * 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.
+ */
+
+
+// Configure artifact push to apache nexus (releases repository).
+
+configure(rootProject) {
+  ext {
+    apacheNexusReleasesRepository = "https://repository.apache.org/service/local/staging/deploy/maven2"
+  }
+
+  // These access credentials must be passed by the release manager
+  // (either on command-line, via the environment or via ~/.gradle.properties).
+  def asfNexusUsername = propertyOrEnvOrDefault('asfNexusUsername', "ASF_NEXUS_USERNAME", null)
+  def asfNexusPassword = propertyOrEnvOrDefault('asfNexusPassword', "ASF_NEXUS_PASSWORD", null)
+
+  task mavenToApacheReleases() {
+    group "Distribution"
+    description "Publish Lucene Maven artifacts to Apache Releases repository: ${apacheNexusReleasesRepository}"
+
+    dependsOn rootProject.ext.mavenProjects.collect {
+      it.tasks.matching { it.name == "publishSignedJarsPublicationToApacheReleasesRepository" }
+    }
+  }
+
+  task checkReleasesRepositoryPushPreconditions() {
+    doFirst {
+      // Make sure we're pushing a release version. The release repository
+      // does not accept snapshots and returns cryptic errors upon trying.
+      if (snapshotBuild) {
+        throw new GradleException("ASF releases repository will not accept a snapshot version: ${rootProject.version}")
+      }
+
+      // Make sure access credentials have been passed.
+      if (asfNexusUsername == null || asfNexusPassword == null) {
+        throw new GradleException("asfNexusUsername or asfNexusPassword is empty: these are required to publish to " +
+            " ASF Nexus.")
+      }
+    }
+  }
+
+  configure(rootProject.ext.mavenProjects) { Project project ->
+    // Make sure any actual publication task is preceded by precondition checks.
+    tasks.matching { it.name ==~ /publish.+ToApacheReleasesRepository/ }.all {
+      dependsOn rootProject.tasks.checkReleasesRepositoryPushPreconditions
+    }
+
+    // Configure the release repository.
+    plugins.withType(PublishingPlugin) {
+      publishing {
+        repositories {
+          maven {
+            name = "ApacheReleases"
+            url = apacheNexusReleasesRepository
+
+            credentials {
+              username asfNexusUsername
+              password asfNexusPassword
+            }
+          }
+        }
+      }
+    }
+  }
+}
\ No newline at end of file
diff --git a/gradle/maven/maven-to-nexus-snapshots.gradle b/gradle/maven/maven-to-nexus-snapshots.gradle
new file mode 100644
index 0000000..cf0fb4a
--- /dev/null
+++ b/gradle/maven/maven-to-nexus-snapshots.gradle
@@ -0,0 +1,76 @@
+/*
+ * 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.
+ */
+
+
+// Configure artifact push to apache nexus (snapshots repository, CI job)
+
+configure(rootProject) {
+  ext {
+    apacheNexusSnapshotsRepository = "https://repository.apache.org/content/repositories/snapshots"
+  }
+
+  // These access credentials are typically passed by the CI job.
+  def asfNexusUsername = propertyOrEnvOrDefault('asfNexusUsername', "ASF_NEXUS_USERNAME", null)
+  def asfNexusPassword = propertyOrEnvOrDefault('asfNexusPassword', "ASF_NEXUS_PASSWORD", null)
+
+  task mavenToApacheSnapshots() {
+    group "Distribution"
+    description "Publish Lucene Maven artifacts to Apache Snapshots repository: ${apacheNexusSnapshotsRepository}"
+
+    dependsOn rootProject.ext.mavenProjects.collect {
+      it.tasks.matching { it.name == "publishJarsPublicationToApacheSnapshotsRepository" }
+    }
+  }
+
+  task checkSnapshotsRepositoryPushPreconditions() {
+    doFirst {
+      // Make sure we're pushing a snapshot version.
+      if (!snapshotBuild) {
+        throw new GradleException("ASF snapshots repository will not accept a non-snapshot version: ${rootProject.version}")
+      }
+
+      // Make sure access credentials have been passed.
+      if (asfNexusUsername == null || asfNexusPassword == null) {
+        throw new GradleException("asfNexusUsername or asfNexusPassword is empty: these are required to publish to " +
+            " ASF Nexus.")
+      }
+    }
+  }
+
+  configure(rootProject.ext.mavenProjects) { Project project ->
+    // Make sure any actual publication task is preceded by precondition checks.
+    tasks.matching { it.name ==~ /publish.+ToApacheSnapshotsRepository/ }.all {
+      dependsOn rootProject.tasks.checkSnapshotsRepositoryPushPreconditions
+    }
+
+    plugins.withType(PublishingPlugin) {
+      publishing {
+        repositories {
+          maven {
+            name = "ApacheSnapshots"
+            url = apacheNexusSnapshotsRepository
+
+            credentials {
+              username asfNexusUsername
+              password asfNexusPassword
+            }
+          }
+        }
+      }
+    }
+  }
+}
\ No newline at end of file
diff --git a/gradle/publishing/defaults-maven.gradle b/gradle/maven/publications-maven.gradle
similarity index 54%
rename from gradle/publishing/defaults-maven.gradle
rename to gradle/maven/publications-maven.gradle
index e5952b5..3ef9461 100644
--- a/gradle/publishing/defaults-maven.gradle
+++ b/gradle/maven/publications-maven.gradle
@@ -15,94 +15,23 @@
  * limitations under the License.
  */
 
+// Configure projects for publishing Maven artifacts and set up repeatable metadata.
 
-// Maven publications and configuration.
-//
-// the 'published' list contains an explicit list of all projects
-// which should be published to Maven repositories.
-
-configure(rootProject) {
-  ext {
-    published = [
-        ":lucene:analysis:common",
-        ":lucene:analysis:icu",
-        ":lucene:analysis:kuromoji",
-        ":lucene:analysis:morfologik",
-        ":lucene:analysis:nori",
-        ":lucene:analysis:opennlp",
-        ":lucene:analysis:phonetic",
-        ":lucene:analysis:smartcn",
-        ":lucene:analysis:stempel",
-        ":lucene:backward-codecs",
-        ":lucene:benchmark",
-        ":lucene:classification",
-        ":lucene:codecs",
-        ":lucene:core",
-        ":lucene:demo",
-        ":lucene:expressions",
-        ":lucene:facet",
-        ":lucene:grouping",
-        ":lucene:highlighter",
-        ":lucene:join",
-        ":lucene:memory",
-        ":lucene:misc",
-        ":lucene:monitor",
-        ":lucene:queries",
-        ":lucene:queryparser",
-        ":lucene:replicator",
-        ":lucene:sandbox",
-        ":lucene:spatial-extras",
-        ":lucene:spatial3d",
-        ":lucene:suggest",
-        ":lucene:test-framework"
-    ]
-
-    apacheNexusSnapshots = "https://repository.apache.org/content/repositories/snapshots"
-  }
-}
+configure(rootProject.ext.mavenProjects) { Project project ->
+  project.apply plugin: 'maven-publish'
+  project.apply plugin: 'signing'
 
-configure(subprojects.findAll { it.path in rootProject.published }) { prj ->
-  apply plugin: 'maven-publish'
-  apply plugin: 'signing'
-
-  publishing {
-    repositories {
-      maven {
-        name = "ApacheSnapshots"
-        url = apacheNexusSnapshots
-
-        credentials {
-          def nexusUserName = rootProject.propertyOrDefault('asfNexusUsername', null)
-          def nexusPwd = rootProject.propertyOrDefault('asfNexusPassword', null)
-          if (nexusUserName && nexusPwd) {
-            username nexusUserName
-            password nexusPwd
-          }
-        }
-      }
-    }
-  }
-
-  // Skip any test fixtures in publishing.
-  afterEvaluate {
-    configurations.matching {
-      return it.name in [
-          "testFixturesApiElements",
-          "testFixturesRuntimeElements"
-      ]
-    }.all {
-      project.components.java.withVariantsFromConfiguration(it) {
-        skip()
+  plugins.withType(JavaPlugin) {
+    // We have two types of publications: jars and signed jars.
+    publishing {
+      publications {
+        jars(MavenPublication)
+        signedJars(MavenPublication)
       }
     }
-  }
 
-  // Do not generate gradle metadata files.
-  tasks.withType(GenerateModuleMetadata) {
-    enabled = false
-  }
-
-  plugins.withType(JavaPlugin) {
+    // Each publication consists of the java components, source and javadoc artifacts.
+    // Add tasks to assemble source and javadoc JARs.
     task sourcesJar(type: Jar, dependsOn: classes) {
       archiveClassifier = 'sources'
       from sourceSets.main.allJava
@@ -113,26 +42,50 @@ configure(subprojects.findAll { it.path in rootProject.published }) { prj ->
       from javadoc.destinationDir
     }
 
-    // This moves publishing configuration after all the scripts of all projects
+    // This moves pom metadata configuration after all the scripts of all projects
     // have been evaluated. This is required because we set artifact groups
-    // and archivesBaseName in other scripts (artifact-naming.gradle) and
-    // maven pom does not accept lazy property providers (so everything must
-    // be in its final form).
-    //
-    // In theory project.afterEvaluate closure should also work but for some reason
-    // it fired earlier than artifact-naming.gradle; don't know whether it's a bug
-    // in gradle or just complex relationships between lazy collection hooks.
+    // and archivesBaseName in other scripts and some of the properties below don't
+    // accept lazy property providers (so everything must be in its final form).
     gradle.projectsEvaluated {
       publishing {
-        def configurePom = {
-            name = "Apache Lucene (module: ${project.name})"
-            description = name
-            url = 'https://lucene.apache.org/'
+        publications.each { publication ->
+          configure(publication) {
+            from components.java
+            groupId = project.group
+            artifactId = project.archivesBaseName
+
+            artifact sourcesJar
+            artifact javadocJar
+
+            // LUCENE-9561:
+            // Remove dependencyManagement section created by a combination of
+            // Palantir and the publishing plugin.
+            //
+            // https://github.com/palantir/gradle-consistent-versions/issues/550
+            pom({
+              withXml {
+                def dm = asNode().dependencyManagement
+                if (dm) dm.replaceNode {}
+              }
+            })
+          }
+        }
+      }
+    }
+
+    // Configure pom defaults for all publications.
+    publishing {
+      publications.each { publication ->
+        configure(publication) {
+          pom {
+            name = provider { -> "Apache Lucene (module: ${project.name})" }
+            description = provider { -> "Apache Lucene (module: ${project.name})" }
+            url = "https://lucene.apache.org/"
 
             licenses {
               license {
                 name = 'Apache 2'
-                url = 'http://www.apache.org/licenses/LICENSE-2.0.txt'
+                url = 'https://www.apache.org/licenses/LICENSE-2.0.txt'
               }
             }
 
@@ -176,51 +129,29 @@ configure(subprojects.findAll { it.path in rootProject.published }) { prj ->
               developerConnection = 'scm:git:https://gitbox.apache.org/repos/asf/lucene.git'
               url = 'https://gitbox.apache.org/repos/asf?p=lucene.git'
             }
-        }
-
-        publications {
-          unsignedJars(MavenPublication)
-          jars(MavenPublication)
-        }
-
-        publications.each { publication ->
-          configure(publication) {
-            from components.java
-            groupId = project.group
-            artifactId = project.archivesBaseName
-
-            artifact sourcesJar
-            artifact javadocJar
-
-            pom(configurePom)
-
-            // LUCENE-9561:
-            // Remove dependencyManagement section created by a combination of
-            // Palantir and the publishing plugin.
-            //
-            // https://github.com/palantir/gradle-consistent-versions/issues/550
-            pom({
-              withXml {
-                def dm = asNode().dependencyManagement
-                if (dm) dm.replaceNode {}
-              }
-            })
           }
         }
       }
+    }
+  }
 
-      // Add aliases of convention tasks with shorter names.
-      task mavenToApacheSnapshots() {
-        group "Publishing"
-        description "Publish Maven JARs and POMs to Apache Snapshots repository: ${apacheNexusSnapshots}"
-
-        dependsOn "publishUnsignedJarsPublicationToApacheSnapshotsRepository"
-      }
-
-      // 'jars' publication is always signed. We currently don't use it anywhere though.
-      signing {
-        sign publishing.publications.jars
+  // Hack: prevent any test fixture JARs from being published.
+  afterEvaluate {
+    configurations.matching { configuration ->
+      configuration.name in [
+          "testFixturesApiElements",
+          "testFixturesRuntimeElements"
+      ]
+    }.all {
+      project.components.java.withVariantsFromConfiguration(it) { variant ->
+        variant.skip()
       }
     }
   }
+
+  // Hack: do not generate or publish gradle metadata files.
+  tasks.withType(GenerateModuleMetadata) {
+    enabled = false
+  }
 }
+
diff --git a/gradle/maven/publications.gradle b/gradle/maven/publications.gradle
new file mode 100644
index 0000000..6ed8b15
--- /dev/null
+++ b/gradle/maven/publications.gradle
@@ -0,0 +1,57 @@
+/*
+ * 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.
+ */
+
+/*
+ * This script configures aspects related to all project publications,
+ * this includes:
+ *
+ * - configuring maven artifacts,
+ * - setting up target maven repositories for publications to end up on,
+ * - configuring binary and source release artifacts,
+ * - other concerns related to publishing artifacts (signing, checksums).
+ */
+
+//
+// An explicit list of projects to publish as Maven artifacts.
+//
+configure(rootProject) {
+  ext {
+    mavenProjects = project(":lucene").subprojects.findAll {subproject ->
+      return !(subproject.path in [
+          // Exclude distribution assembly & documentation.
+          ":lucene:distribution",
+          ":lucene:documentation",
+          // Exclude the parent container project for analysis modules (no artifacts).
+          ":lucene:analysis",
+          // Exclude the native module.
+          ":lucene:misc:native"
+      ])
+    }
+  }
+}
+
+// Configure projects for publishing Maven artifacts and set up metadata.
+apply from: buildscript.sourceFile.toPath().resolveSibling("publications-maven.gradle")
+
+// Configure on-demand maven publishing into ~/.m2 for developers' convenience.
+apply from: buildscript.sourceFile.toPath().resolveSibling("maven-to-local-m2.gradle")
+
+// Configure artifact push to apache nexus (snapshots repository, CI job).
+apply from: buildscript.sourceFile.toPath().resolveSibling("maven-to-nexus-snapshots.gradle")
+
+// Configure artifact push to apache nexus (releases repository).
+apply from: buildscript.sourceFile.toPath().resolveSibling("maven-to-nexus-releases.gradle")
diff --git a/gradle/publishing/distribution.gradle b/gradle/publishing/distribution.gradle
deleted file mode 100644
index 407ead4..0000000
--- a/gradle/publishing/distribution.gradle
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.
- */
-
-import org.apache.commons.codec.digest.DigestUtils
-
-allprojects {
-    plugins.withType(DistributionPlugin) {
-        def checksum = {
-            outputs.files.each { File file ->
-                String sha512 = new DigestUtils(DigestUtils.sha512Digest).digestAsHex(file).trim()
-                // The asterisk signals a binary file, see https://www.gnu.org/software/coreutils/manual/coreutils.html#md5sum-invocation
-                new File(file.parent, file.name + ".sha512").write(sha512 + " *" + file.name, "UTF-8")
-            }
-        }
-
-        distZip {
-            doLast checksum
-        }
-
-        distTar {
-            compression = Compression.GZIP
-            doLast checksum
-        }
-
-        installDist {
-            doLast {
-                logger.lifecycle "Distribution assembled under: ${destinationDir}"
-            }
-        }
-    }
-}
diff --git a/gradle/publishing/maven-local.gradle b/gradle/publishing/maven-local.gradle
deleted file mode 100644
index e3d8268..0000000
--- a/gradle/publishing/maven-local.gradle
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.
- */
-
-
-// This adds a root project task to install all artifacts to a build-local
-// Maven repository (so that pom files can be manually reviewed).
-
-configure(rootProject) {
-  ext {
-    mavenLocalDir = file("${buildDir}/maven-local")
-  }
-
-  task mavenToLocalFolder() {
-    group "Publishing"
-    description "Publish Maven JARs and POMs locally to " + mavenLocalDir
-
-    doLast {
-      logger.lifecycle "Local maven artifacts (poms, jars) created at: ${mavenLocalDir}"
-    }
-  }
-
-  task mavenToLocalRepo() {
-    group "Publishing"
-    description "Publish Maven JARs and POMs to current user's local maven repository."
-  }
-
-  task mavenLocalClean(type: Delete) {
-    delete mavenLocalDir
-  }
-
-  configure(subprojects.findAll { it.path in rootProject.published }) {
-    plugins.withType(PublishingPlugin) {
-      publishing {
-        repositories {
-          maven {
-            name = 'build'
-            url = mavenLocalDir
-          }
-        }
-      }
-
-      tasks.matching { it.name == "publishUnsignedJarsPublicationToMavenLocal" }.all { task ->
-        mavenToLocalRepo.dependsOn task
-      }
-
-      tasks.matching { it.name == "publishUnsignedJarsPublicationToBuildRepository" }.all { task ->
-        // Clean local repo prior to republishing artifacts.
-        task.dependsOn mavenLocalClean
-        // Attach to root project's mavenLocal task.
-        mavenToLocalFolder.dependsOn task
-      }
-    }
-  }
-}
\ No newline at end of file
diff --git a/help/gpgSigning.txt b/help/gpgSigning.txt
deleted file mode 100644
index 80e45f0..0000000
--- a/help/gpgSigning.txt
+++ /dev/null
@@ -1,70 +0,0 @@
-GPG Signing
-===========
-
-GPG Signing of distribution files (typically by a release manager) is done with the 'signDist' command.
-
-The only required configuration property Gradle needs is the 'signing.gnupg.keyName' (aka: the fingerprint) of
-the key you wish to use:
-
-./gradlew signDist -Psigning.gnupg.keyName=YOUR_KEY_FINGERPRINT
-
-By default when you run this command, Gradle will delegate to the `gpg2` command for managing the signing of each file, which (should)
-in turn use the `gpg-agent` to prompt you for your secret key only as needed using a dialog box specific to your operating system and/or
-`gpg-agent` preferences.
-
-You may wish to put the `signing.gnupg.keyName` in your `~/.gradle/gradle.properties` so it is set automatically any time you use gradle
-
-
-Additional Configuration
-------------------------
-
-The following additional properties -- specified either on the command line via `-P...` or in your `~/.gradle/gradle.properties` may be
-useful/necessary in your system:
-
-signing.gnupg.useLegacyGpg=true                    # Changes the default executable from `gpg2` to `gpg` and explicitly sets `--use-agent`
-signing.gnupg.executable=gpg                       # Allows explicit control over what command executable used (ex: `gpg2`, `gpg`, `gpg.exe`, etc...)
-signing.gnupg.homeDir=/tmp/gnupg-home              # overrides GnuPG's default home directory (ex: `~/.gnupg/`)
-signing.gnupg.optionsFile=/tmp/gnupg-home/my.conf  # overrides GnuPG's default configuration file
-signing.gnupg.passphrase=...                       # Provide your passphrase to gradle to hand off to gpg.  *NOT RECOMMENDED*, see below.
-
-
-Notes About Error Messages
---------------------------
-
-
-### `gpg: signing failed: Inappropriate ioctl for device`
-
-This typically happens if your `gpg-agent` is configured (either globally for your operating system, or personally in your
-`~/.gnupg/gpg-agent.conf`) to use a `pinentry` command which depends on using the same `tty` as the `gpg` command (ex: `pinentry-curses`,
-or `pinentry-tty`, etc...).
-
-`tty` based `pinentry` implementations do not work when Gradle's `SigningPlugin` is attempting to invoke `gpg` -- among other problems:
-Gradle is multi-threaded, and we sign multiple artifacts by default; so even if the `SigningPlugin` didn't automatically force `--no-tty` when
-running `gpg` you could easily run into problems where a second `pinentry` process wanted to read from the same `tty` in the middle of you
-typing in your passphrase to the first process.
-
-Developers are encouraged to configure a *non* `tty` based `pinentry` (ex: `pinentry-gnome`, `pinentry-x11`, `pinentry-qt`, `pinentry-mac`,
-`pinentry-wsl-ps1`, etc...) either globally in your operating system, or personally in your `~/.gnupg/gpg-agent.conf`, or in a new
-`gpg-agent.conf` file a new GnuPG configuration directory (containing a copy of your private keys) that you direct gradle to via
-`signing.gnupg.homeDir`
-
-If none of these options are viable for you, then as a last resort you may wish to consider using the `signing.gnupg.passphrase=...` property.
-This will expose your secret passphrase to the Gradle process, which will then pass it directly to each `gpg-agent` instance using
-`--pinentry-mode=loopback`.
-
-
-### `gpg: signing failed: No such file or directory`
-
-This may mean that there is a problem preventing `gpg` from communicating correctly with the `gpg-agent` (and/or invoking your `pinentry`
-program) that is independent of gradle.  Try running `pkill gpg-agent` and then retrying your `./gradlew` command
-
-
-### `No value has been specified for property 'signatory.keyId'.`
-
-Do not bother ever attempting to set a command line (or gradle.properties) property named `signatory.keyId`. This is evidently the
-name of an internal property that the gradle `SigningPlugin` expects the `GnupgSignatory` plugin we use to provide -- which it does
-as long as you have specified a valid value for `signing.gnupg.keyName`
-
-If you see this error, it means you did not properly set `signing.gnupg.keyName` _AND_ you invoked a task which is attempting to use
-the `SigningPlugin`, but does not depend on the custom `failUnlessGpgKeyProperty` to report the error correctly.  Please file a Jira
-noting what `./gradlew` command you attempted to run so we can fix it's dependencies, and try again after setting `signing.gnupg.keyName`.
diff --git a/help/publishing.txt b/help/publishing.txt
new file mode 100644
index 0000000..86852cf
--- /dev/null
+++ b/help/publishing.txt
@@ -0,0 +1,134 @@
+Distribution and artifact publishing
+====================================
+
+
+See all distribution-related tasks by running:
+gradlew tasks --group distribution
+
+
+Maven
+-----
+
+To publish Lucene Maven artifacts to a local ~/.m2 repository, run:
+
+gradlew mavenToLocal
+
+To publish Lucene Maven artifacts to Apache repositories
+(CI or release manager's job, typically!), run:
+
+gradlew mavenToApacheSnapshots -PasfNexusUsername= -PasfNexusPassword=
+gradlew mavenToApacheReleases  -PasfNexusUsername= -PasfNexusPassword= [optional signing options]
+
+See artifact signing section below if you plan to use mavenToApacheReleases.
+
+It is a good idea to avoid passing passwords on command line. CI jobs have
+these properties saved in ~/.gradle/gradle.properties - this way they
+are read automatically.
+
+Apache Releases repository will not accept snapshots.
+
+
+Release (distribution) artifacts
+--------------------------------
+
+To collect all release artifacts, and optionally sign them, run:
+
+gradlew assembleRelease [optional signing options]
+
+All distribution artifacts will be placed under:
+
+lucene/distribution/build/release
+
+Artifact signing is optional (but required if you're really making a release).
+
+
+Artifact signing
+----------------
+
+Certain tasks may optionally sign artifacts or require artifacts to be signed:
+
+ assembleRelease
+ mavenToApacheReleases
+
+Signing can be enabled by adding the "-Psign" option, for example:
+
+gradlew assembleRelease mavenToApacheReleases -Psign
+
+If using signatures, make yourself familiar with how to pass the required signatory
+credentials via ~/.gradle/gradle.properties or command-line options:
+
+https://docs.gradle.org/current/userguide/signing_plugin.html#sec:signatory_credentials
+
+An example full command-line that assembles signed artifacts could look like this:
+
+gradlew assembleRelease mavenToApacheReleases -Psign -Psigning.keyId=... -Psigning.password=... -Psigning.secretKeyRingFile=...
+
+The keyId is the last 8 digits of your key (gpg -k will print your keys). Gradle documentation has more options
+of secure passing of private key information and passwords.
+
+
+Artifact signing using an external GPG (or GPG agent)
+-----------------------------------------------------
+
+You can use an external GPG command (or GPG agent) but this changes the options used and may require a
+restriction on gradle concurrency:
+
+For gpg2:
+gradlew [tasks] -Psign -PuseGpg --max-workers 1 -Psigning.gnupg.keyName=...
+
+For gpg:
+gradlew [tasks] -Psign -PuseGpg --max-workers 1 -Psigning.gnupg.keyName=... -Psigning.gnupg.useLegacyGpg=true
+
+The keyName is the last 8 digits of your key (gpg -k will print your keys). 
+
+There are a few possible quirks when using an external GPG or GPG agent. 
+The following additional properties -- specified either on the command line via `-P...` 
+or in your `~/.gradle/gradle.properties` may be handy:
+
+signing.gnupg.useLegacyGpg=true                    # Changes the default executable from `gpg2` to `gpg` and explicitly sets `--use-agent`
+signing.gnupg.executable=gpg                       # Allows explicit control over what command executable used (ex: `gpg2`, `gpg`, `gpg.exe`, etc...)
+signing.gnupg.homeDir=/tmp/gnupg-home              # overrides GnuPG's default home directory (ex: `~/.gnupg/`)
+signing.gnupg.optionsFile=/tmp/gnupg-home/my.conf  # overrides GnuPG's default configuration file
+signing.gnupg.passphrase=...                       # Provide your passphrase to gradle to hand off to gpg.  *NOT RECOMMENDED*, see below.
+
+If in doubt, consult gradle's signing plugin documentation:
+https://docs.gradle.org/current/userguide/signing_plugin.html#sec:using_gpg_agent
+
+
+Notes About GPG Error Messages
+------------------------------
+
+### `gpg: signing failed: Inappropriate ioctl for device`
+
+This typically happens if your `gpg-agent` is configured (either globally for your operating system, or personally in your
+`~/.gnupg/gpg-agent.conf`) to use a `pinentry` command which depends on using the same `tty` as the `gpg` command (ex: `pinentry-curses`,
+or `pinentry-tty`, etc...).
+
+`tty` based `pinentry` implementations do not work when Gradle's `SigningPlugin` is attempting to invoke `gpg` -- among other problems:
+Gradle is multi-threaded (hence --max-workers 1 above to force single-threaded execution), and we sign multiple artifacts by 
+default; so even if the `SigningPlugin` didn't automatically force `--no-tty` when running `gpg`, you could easily run into problems 
+where a second `pinentry` process wanted to read from the same `tty` in the middle of you typing in your passphrase to the first process.
+
+Developers are encouraged to configure a *non* `tty` based `pinentry` (ex: `pinentry-gnome`, `pinentry-x11`, `pinentry-qt`, `pinentry-mac`,
+`pinentry-wsl-ps1`, etc...) either globally in your operating system, or personally in your `~/.gnupg/gpg-agent.conf`, or in a new
+`gpg-agent.conf` file a new GnuPG configuration directory (containing a copy of your private keys) that you direct gradle to via
+`signing.gnupg.homeDir`
+
+If none of these options are viable for you, then as a last resort you may wish to consider using the `signing.gnupg.passphrase=...` property.
+This will expose your secret passphrase to the Gradle process, which will then pass it directly to each `gpg-agent` instance using
+`--pinentry-mode=loopback`.
+
+
+### `gpg: signing failed: No such file or directory`
+
+This may mean that there is a problem preventing `gpg` from communicating correctly with the `gpg-agent` (and/or invoking your `pinentry`
+program) that is independent of gradle.  Try running `pkill gpg-agent` and then retrying your `./gradlew` command
+
+
+### `No value has been specified for property 'signatory.keyId'.`
+
+This typically means something went wrong when communicating with the external GPG. This is the
+name of an internal property that the gradle's `SigningPlugin` expects in non-GPG mode. The error message is just confusing.
+
+If you see this error, it means you did not properly set `signing.gnupg.keyName` _AND_ you invoked a task which is attempting to use
+the `SigningPlugin`. Please file a Jira issue and describe the problem, maybe there is a workaround for it. 
diff --git a/lucene/distribution/artifact-signing.gradle b/lucene/distribution/artifact-signing.gradle
new file mode 100644
index 0000000..d0bf6d3
--- /dev/null
+++ b/lucene/distribution/artifact-signing.gradle
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+
+// Configure release artifact signing.
+
+task signReleaseArchives(type: Sign) {
+  dependsOn tasks.assembleBinaryZip
+  sign tasks.assembleBinaryZip
+
+  dependsOn tasks.assembleBinaryTgz
+  sign tasks.assembleBinaryTgz
+
+  // The source tgz is not an archive task so be explicit about the outputs to sign.
+  dependsOn tasks.assembleSourceTgz
+  sign tasks.assembleSourceTgz.destination
+}
+
+
+// Optionally, switch to using GPG command (or agent). This entails some additional
+// oddities so add some extra sanity checks.
+if (propertyOrDefault("useGpg", null) != null) {
+  signing {
+    useGpgCmd()
+  }
+
+  if (gradle.startParameter.maxWorkerCount != 1) {
+    throw new GradleException("When using GPG for signing, specify --max-workers 1 to ensure sequential GPG calls.")
+  }
+}
diff --git a/lucene/distribution/binary-artifacts.gradle b/lucene/distribution/binary-artifacts.gradle
new file mode 100644
index 0000000..15d83cd
--- /dev/null
+++ b/lucene/distribution/binary-artifacts.gradle
@@ -0,0 +1,73 @@
+/*
+ * 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.
+ */
+
+// Configure binary distribution content defaults for each subproject.
+
+configure(rootProject.ext.mavenProjects) {
+  plugins.withType(JavaPlugin) {
+    ext {
+      binaryArtifactsDir = file("${buildDir}/binaryArtifacts")
+    }
+
+    configurations {
+      // All of the module's "binary" artifacts that should go to the release.
+      binaryArtifacts
+
+      // Legacy binary distribution contains Lucene artifacts (JARs) and their dependencies
+      // under 'lib/'. We have to go through some hoops to split the artifact from all of its
+      // transitive dependencies and separate their output location.
+      transitiveDependencies {
+        // All of runtime elements
+        extendsFrom runtimeElements
+        // Exclude cross-module dependencies
+        exclude group: "org.apache.lucene"
+        // Exclude the following from all projects, if they appear.
+        exclude group: "commons-logging"
+        exclude group: "org.slf4j"
+      }
+    }
+
+    task assembleBinaryArtifacts(type: Sync) {
+      from(tasks.findByName("jar"))
+
+      from(projectDir, {
+        include "README.*"
+      })
+
+      from(configurations.transitiveDependencies, {
+        into "lib/"
+      })
+
+      into binaryArtifactsDir
+    }
+
+    artifacts {
+      binaryArtifacts binaryArtifactsDir, {
+        builtBy assembleBinaryArtifacts
+      }
+    }
+  }
+}
+
+
+// Add launch scripts for Luke.
+configure(project(":lucene:luke")) {
+  tasks.matching { it.name == "assembleBinaryArtifacts" }.all {
+    from("${projectDir}/bin", {
+    })
+  }
+}
diff --git a/lucene/distribution/binary-release.gradle b/lucene/distribution/binary-release.gradle
new file mode 100644
index 0000000..34bc416
--- /dev/null
+++ b/lucene/distribution/binary-release.gradle
@@ -0,0 +1,125 @@
+/*
+ * 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.
+ */
+
+
+// Configure Lucene's binary release. This is a bit convoluted so is placed
+// in a separate script.
+
+configure(project(":lucene:distribution")) {
+  def packageBaseName = "${buildDir}/packages/lucene-${version}"
+
+  // All the maven-published projects are part of the binary distribution.
+  def includeInBinaries = rootProject.ext.mavenProjects
+
+  // Legacy binary distribution contains Lucene artifacts (JARs) and their dependencies
+  // under 'lib/'. We have to go through some hoops to split the artifact from all of its
+  // transitive dependencies and separate their output location.
+  def moduleArtifactConfigurations = []
+  for (Project module : includeInBinaries) {
+    def binaryArtifactsConfiguration = configurations.create("binary-artifacts" + module.path.replace(':', '-'))
+
+    dependencies { DependencyHandler handler ->
+      handler.add(binaryArtifactsConfiguration.name, project(path: module.path, configuration: "binaryArtifacts"))
+    }
+
+    moduleArtifactConfigurations += [
+        "module": module,
+        "binaryArtifactsConfiguration": binaryArtifactsConfiguration
+    ]
+  }
+
+
+  // Prepare site documentation dependency for inclusion.
+  configurations {
+    docs
+  }
+
+  dependencies {
+    docs project(path: ':lucene:documentation', configuration: 'site')
+  }
+
+
+  // Distribution package (zip/tgz) content is the same so use the same closure to configure both tasks.
+  Closure<Void> distributionContent = { AbstractCopyTask task ->
+    // Manually correct posix permissions (matters when assembling archives on Windows).
+    filesMatching(["**/*.sh", "**/*.bat"]) { copy ->
+      copy.setMode(0755)
+    }
+
+    // Cherry-picked root-level files.
+    from(project(':lucene').projectDir, {
+      include "CHANGES.txt"
+      include "JRE_VERSION_MIGRATION.md"
+      include "LICENSE.txt"
+      include "MIGRATE.md"
+      include "NOTICE.txt"
+      include "README.md"
+      include "SYSTEM_REQUIREMENTS.md"
+
+      include "licenses/*"
+    })
+
+    // Analysis container is not a submodule but include its readme file.
+    from(project(':lucene:analysis').projectDir) {
+      include "README.txt"
+      into 'analysis'
+    }
+
+    // The documentation.
+    from(configurations.docs, {
+      into 'docs'
+    })
+
+    // Each module's binary artifacts and their dependencies.
+    moduleArtifactConfigurations.each {
+      def toPath = it.module.path.replaceFirst("^:lucene:", "").replace(':', '/')
+
+      task.from(it.binaryArtifactsConfiguration, {
+        into toPath
+      })
+    }
+
+    // Internal archive folder for all files.
+    into "lucene-${rootProject.version}/"
+  }
+
+  // Assemble distribution archives
+  task assembleBinaryZip(type: Zip) {
+    description "Assemble binary Lucene artifact as a .zip file."
+
+    archiveFileName = packageBaseName + ".zip"
+    destinationDirectory = file(archiveFileName).parentFile
+
+    metadataCharset = "UTF-8"
+    reproducibleFileOrder = true
+  }
+
+  task assembleBinaryTgz(type: Tar) {
+    description "Assemble binary Lucene artifact as a .tgz file."
+
+    archiveFileName = packageBaseName + ".tgz"
+    destinationDirectory = file(archiveFileName).parentFile
+
+    reproducibleFileOrder = true
+    compression = Compression.GZIP
+  }
+
+  // Apply the content configuration.
+  [tasks.assembleBinaryZip, tasks.assembleBinaryTgz].each { Task task ->
+    task.configure distributionContent
+  }
+}
diff --git a/lucene/distribution/build.gradle b/lucene/distribution/build.gradle
new file mode 100644
index 0000000..4bdbebf
--- /dev/null
+++ b/lucene/distribution/build.gradle
@@ -0,0 +1,109 @@
+/*
+ * 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.
+ */
+
+import org.apache.lucene.gradle.Checksum
+
+plugins {
+  id 'signing'
+}
+
+// This project puts together the Lucene "distribution", assembling bits and pieces
+// from across the project structure into release artifacts.
+
+ext {
+  releaseDir = file("${buildDir}/release")
+}
+
+// Prepare the "source" distribution artifact.
+apply from: buildscript.sourceFile.toPath().resolveSibling("source-release.gradle")
+
+// Prepare the "binary" distribution artifact.
+apply from: buildscript.sourceFile.toPath().resolveSibling("binary-artifacts.gradle")
+apply from: buildscript.sourceFile.toPath().resolveSibling("binary-release.gradle")
+
+// Configure maven artifact collection to a local build folder (required to collect artifacts for the release).
+apply from: buildscript.sourceFile.toPath().resolveSibling("collect-maven-artifacts.gradle")
+
+// Configure artifact signing.
+apply from: buildscript.sourceFile.toPath().resolveSibling("artifact-signing.gradle")
+
+
+// Set up the HTML-rendered "changes" distribution artifact by linking to documentation's output.
+configurations {
+  changesHtml
+}
+
+dependencies {
+  changesHtml project(path: ":lucene:documentation", configuration: "changesHtml")
+}
+
+
+// Compute checksums for release archives.
+task computeChecksums(type: Checksum) {
+  algorithm = Checksum.Algorithm.SHA512
+
+  files = objects.fileCollection()
+  [
+      tasks.assembleSourceTgz,
+      tasks.assembleBinaryZip,
+      tasks.assembleBinaryTgz,
+  ].each { dep ->
+    dependsOn dep
+    files += dep.outputs.files
+  }
+
+  outputDir = file("${buildDir}/checksums")
+}
+
+
+// Assemble everything needed in the release folder structure.
+task assembleRelease(type: Sync) {
+  description "Assemble all Lucene artifacts for a release."
+
+  from(configurations.changesHtml, {
+    into "changes"
+  })
+
+  from(tasks.mavenToBuild, {
+    into "maven"
+  })
+
+  from tasks.assembleSourceTgz
+  from tasks.assembleBinaryZip
+  from tasks.assembleBinaryTgz
+
+  from tasks.computeChecksums
+
+  // Conditionally, attach signatures of all the release archives.
+  if (propertyOrDefault("sign", null) != null) {
+    from tasks.signReleaseArchives
+  }
+
+  into releaseDir
+}
+
+
+// Add the description and task group to some of the tasks that make
+// sense at the user-level help.
+tasks.matching {it.name in [
+    "assembleSourceTgz",
+    "assembleBinaryZip",
+    "assembleBinaryTgz",
+    "assembleRelease",
+]}.all {
+  group "distribution"
+}
diff --git a/lucene/distribution/collect-maven-artifacts.gradle b/lucene/distribution/collect-maven-artifacts.gradle
new file mode 100644
index 0000000..e580e72
--- /dev/null
+++ b/lucene/distribution/collect-maven-artifacts.gradle
@@ -0,0 +1,58 @@
+/*
+ * 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.
+ */
+
+
+//
+// Configure artifact publishing into a build-local Maven repository. This is technically not required
+// at all - maven artifacts can go directly to ASF Nexus - but smoke-testing scripts run checks on these
+// artifacts.
+//
+
+configure(project(":lucene:distribution")) {
+  def mavenRepositoryDir = file("${buildDir}/maven-artifacts")
+
+  task mavenToBuild() {
+    outputs.dir(mavenRepositoryDir)
+
+    dependsOn rootProject.ext.mavenProjects.collect {
+      it.tasks.matching { it.name == "publishJarsPublicationToBuildRepository" }
+    }
+  }
+
+  def cleanBuildTask = task cleanMavenBuildRepository(type: Delete) {
+    delete mavenRepositoryDir
+  }
+
+  configure(rootProject.ext.mavenProjects) { Project project ->
+    // Clean the build repository prior to publishing anything. Ensures we don't
+    // have multiple artifacts there.
+    tasks.matching { it.name ==~ /publish.+ToBuildRepository/ }.all {
+      dependsOn cleanBuildTask
+    }
+
+    plugins.withType(PublishingPlugin) {
+      publishing {
+        repositories {
+          maven {
+            name = "Build"
+            url = mavenRepositoryDir
+          }
+        }
+      }
+    }
+  }
+}
diff --git a/lucene/distribution/source-release.gradle b/lucene/distribution/source-release.gradle
new file mode 100644
index 0000000..fecb435
--- /dev/null
+++ b/lucene/distribution/source-release.gradle
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+
+
+// Prepare the "source" distribution artifact. We use raw git export, no additional complexity needed.
+
+configure(project(":lucene:distribution")) {
+  task assembleSourceTgz() {
+    description "Assemble source Lucene artifact as a .tgz file."
+
+    // Make sure no unstaged/ dirty files are present.
+    dependsOn ":gitStatus", ":checkWorkingCopyClean"
+
+    ext {
+      destination = file("${buildDir}/packages/lucene-${version}-src.tgz")
+    }
+
+    inputs.property("git-revision", { -> rootProject.ext.gitRev })
+    outputs.file destination
+
+    doFirst {
+      quietExec {
+        executable = project.externalTool("git")
+        workingDir = project.rootDir
+
+        args += [
+            "archive",
+            "--format", "tgz",
+            "--prefix", "lucene-${version}/",
+            "--output", destination,
+            "HEAD"
+        ]
+      }
+    }
+  }
+}
diff --git a/lucene/packaging/build.gradle b/lucene/packaging/build.gradle
deleted file mode 100644
index 1a701d4..0000000
--- a/lucene/packaging/build.gradle
+++ /dev/null
@@ -1,256 +0,0 @@
-/*
- * 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.
- */
-
-// This project puts together a "distribution", assembling dependencies from
-// various other projects.
-
-plugins {
-    id 'distribution'
-    id 'signing'
-}
-
-description = 'Lucene distribution packaging'
-
-// Declare all subprojects that should be included in binary distribution.
-// By default everything is included, unless explicitly excluded.
-def includeInBinaries = project(":lucene").subprojects.findAll {subproject ->
-    return !(subproject.path in [
-        // Exclude packaging & documentation, not relevant to binary distribution.
-        ":lucene:packaging",
-        ":lucene:documentation",
-        // Exclude parent container project of analysis modules (no artifacts).
-        ":lucene:analysis",
-        // Exclude native module, which requires manual copying and enabling
-        ":lucene:misc:native"
-    ])
-}
-
-// Create a configuration to each subproject and add dependency.
-def binaryArtifactsConf = { Project prj ->
-    "dep-binary" + prj.path.replace(':', '-')
-}
-
-def allDepsConf = { Project prj ->
-    "dep-full" + prj.path.replace(':', '-')
-}
-
-configurations {
-    docs
-}
-
-for (Project includedProject : includeInBinaries) {
-    def confBinaries = binaryArtifactsConf(includedProject)
-    def confFull = allDepsConf(includedProject)
-    configurations.create(confBinaries)
-    configurations.create(confFull)
-    dependencies { DependencyHandler handler ->
-        // Just project binaries.
-        handler.add(confBinaries, project(path: includedProject.path, configuration: "packaging"))
-        // All project dependencies, including transitive dependencies from the runtime configuration.
-        handler.add(confFull, project(path: includedProject.path, configuration: "runtimeElements"), {
-            exclude group: "org.apache.lucene"
-
-            // Exclude these from all projects.
-            exclude group: "commons-logging"
-            exclude group: "org.slf4j"
-        })
-    }
-}
-
-dependencies {
-    docs project(path: ':lucene:documentation', configuration: 'site')
-}
-
-distributions {
-    // The "main" distribution is the binary distribution.
-    // We should also add 'source' distribution at some point
-    // (we can't do it now as the build itself is tangled with Solr).
-    main {
-        distributionBaseName = 'lucene'
-
-        contents {
-            // Manually correct posix permissions (matters when packaging on Windows).
-            filesMatching(["**/*.sh", "**/*.bat"]) { copy ->
-                copy.setMode(0755)
-            }
-
-            // Root distribution files; these are cherry-picked manually.
-            from(project(':lucene').projectDir, {
-                include "CHANGES.txt"
-                include "JRE_VERSION_MIGRATION.md"
-                include "LICENSE.txt"
-                include "licenses/*"
-                include "MIGRATE.md"
-                include "NOTICE.txt"
-                include "README.md"
-                include "SYSTEM_REQUIREMENTS.md"
-            })
-
-            // A couple more missing README files
-            from(project(':lucene:analysis').projectDir) {
-                include "README.txt"
-                into 'analysis'
-            }
-
-            // Copy files from documentation output to 'docs'
-            from(configurations.docs, {
-                into 'docs'
-            })
-
-            // Put included project's binary artifacts under their corresponding subfolders.
-            // This needs to be a for-loop instead of projects.each since the "from" could be resolved at the wrong time
-            for (Project includedProject : includeInBinaries) {
-                def basePath = includedProject.path.replaceFirst("^:lucene:", "").replace(':', '/')
-                def confBinaries = configurations.findByName(binaryArtifactsConf(includedProject))
-                def confFull = configurations.findByName(allDepsConf(includedProject))
-
-                from(confBinaries, {
-                    into basePath
-                })
-                from(confFull, {
-                    into "${basePath}/lib"
-                    // Manually exclude the project's main artifact. Don't know if there is any better way to do this.
-                    exclude "lucene-*"
-                })
-            }
-        }
-    }
-}
-
-// Add common packaging artifacts.
-configure(project(":lucene").subprojects) {
-    plugins.withType(JavaPlugin) {
-        ext {
-            packagingDir = file("${buildDir}/packaging")
-        }
-
-        configurations {
-            packaging
-        }
-
-        task assemblePackaging(type: Sync) {
-            from(projectDir, {
-                include "README.*"
-            })
-
-            from(tasks.findByName("jar"))
-
-            into packagingDir
-        }
-
-        artifacts {
-            packaging packagingDir, {
-                builtBy assemblePackaging
-            }
-        }
-    }
-}
-
-// Add launch scripts for Luke.
-configure(project(":lucene:luke")) {
-    plugins.withType(JavaPlugin) {
-        assemblePackaging {
-            from("${projectDir}/bin", {
-            })
-        }
-    }
-}
-
-configurations {
-  luceneTgz
-  luceneZip
-}
-artifacts {
-  luceneTgz(distTar)
-  luceneZip(distZip)
-}
-
-// Source distribution using git export
-def sourceTarFile = file("build/distributions/lucene-${version}-src.tgz")
-import org.apache.commons.codec.digest.DigestUtils
-
-task assembleSourceDist() {
-    def target = sourceTarFile
-
-    outputs.files target
-
-    // TODO: This is copied from distribution.gradle - reuse?
-    def checksum = { file ->
-        String sha512 = new DigestUtils(DigestUtils.sha512Digest).digestAsHex(file).trim()
-        // The asterisk signals a binary file, see https://www.gnu.org/software/coreutils/manual/coreutils.html#md5sum-invocation
-        new File(file.parent, file.name + ".sha512").write(sha512 + " *" + file.name, "UTF-8")
-    }
-
-    doFirst {
-        quietExec {
-            executable = project.externalTool("git")
-            workingDir = project.rootDir
-
-            args += [
-                    "archive",
-                    "--format", "tgz",
-                    "--prefix", "lucene-${version}/",
-                    "--output", target,
-                    "HEAD"
-            ]
-        }
-        checksum(sourceTarFile)
-    }
-}
-assembleDist.dependsOn assembleSourceDist
-
-// NOTE: we don't use the convinence DSL of the 'signing' extension to define our 'Sign' tasks because
-// that (by default) adds our signature files to the 'archives' configuration -- which is what
-// assemble & installDist try to copy/sync, so they wouldn't work w/o GPG installed (which would be bad).
-//
-// We also want to hook in our own property check dependency since the default error message from Sign task
-// refers to the wrong (internal only) property name ("signatory.keyId")
-signing {
-  useGpgCmd() // so gpg-agent can be used
-}
-task failUnlessGpgKeyProperty {
-  // placeholder that can be depended on by any task needing GPG key which will 'fail fast' if it's not set.
-  def propName = 'signing.gnupg.keyName'
-
-  // This explicitly checks the taskGraph (instead of a simple 'doFirst') so it can fail the user's gradle
-  // invocation immediately before any unrelated build tasks may run in parallel
-  if ( ! project.hasProperty(propName) ) {
-    gradle.taskGraph.whenReady { graph ->
-      if ( graph.hasTask(failUnlessGpgKeyProperty) ) {
-        // TODO: it would be really nice if taskGraph was an actual graph and we could report what tasks in (transitive) depend on us
-        throw new GradleException("'$propName' property must be set for GPG signing, please see help/gpgSigning.txt")
-      }
-    }
-  }
-}
-task signDistTar(type: Sign) {
-  dependsOn failUnlessGpgKeyProperty
-  sign configurations.luceneTgz
-}
-task signDistZip(type: Sign) {
-  dependsOn failUnlessGpgKeyProperty
-  sign configurations.luceneZip
-}
-task signSourceDistTar(type: Sign) {
-    dependsOn failUnlessGpgKeyProperty, assembleSourceDist
-    sign sourceTarFile
-}
-task signDist {
-  group = 'Distribution'
-  description = 'GPG Signs the main distributions'
-  dependsOn signDistTar, signDistZip, signSourceDistTar
-}
diff --git a/settings.gradle b/settings.gradle
index 90d8957..e965095 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -60,4 +60,4 @@ include "lucene:spatial3d"
 include "lucene:suggest"
 include "lucene:test-framework"
 include "lucene:documentation"
-include "lucene:packaging"
+include "lucene:distribution"