You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2019/06/06 05:44:30 UTC

[lucene-solr] 04/04: SOLR-13452: Some cleanup, add a new task to help find unused deps, clean up some deps.

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

markrmiller pushed a commit to branch jira/SOLR-13452_gradle_3
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit b74abe272d58378626596de077c1de5e1f39c662
Author: markrmiller <ma...@apache.org>
AuthorDate: Thu Jun 6 00:22:47 2019 -0500

    SOLR-13452: Some cleanup, add a new task to help find unused deps, clean up some deps.
---
 build.gradle                                       |   3 +-
 .../org/apache/lucene/gradle/JarChecksum.groovy    |   2 +-
 .../org/apache/lucene/gradle/UnusedDeps.groovy     | 138 +++++++++++++++++++++
 lucene/analysis/common/build.gradle                |   2 +-
 lucene/analysis/icu/build.gradle                   |   2 +-
 lucene/analysis/kuromoji/build.gradle              |   2 +-
 lucene/analysis/morfologik/build.gradle            |   2 +-
 lucene/analysis/nori/build.gradle                  |   2 +-
 lucene/analysis/opennlp/build.gradle               |   2 +
 lucene/analysis/phonetic/build.gradle              |   2 +-
 lucene/analysis/smartcn/build.gradle               |   2 +-
 lucene/analysis/stempel/build.gradle               |   2 +-
 lucene/build.gradle                                |   7 +-
 lucene/expressions/build.gradle                    |   1 -
 solr/build.gradle                                  |   5 +
 solr/contrib/analytics/build.gradle                |   3 -
 solr/contrib/clustering/build.gradle               |  19 ++-
 solr/contrib/ltr/build.gradle                      |   1 -
 solr/contrib/prometheus-exporter/build.gradle      |   1 -
 solr/core/build.gradle                             |  58 ++++-----
 solr/solrj/build.gradle                            |  10 +-
 solr/test-framework/build.gradle                   |   4 +-
 versions.lock                                      |  39 +++---
 23 files changed, 219 insertions(+), 90 deletions(-)

diff --git a/build.gradle b/build.gradle
index 617cf22..496aa6b 100644
--- a/build.gradle
+++ b/build.gradle
@@ -171,9 +171,10 @@ configure(allprojects) {
 
     dependencies {
       configurations.all {
-        // these classes are in Java 11
+        // stax-api classes are in Java 11
         exclude group: 'javax.xml.stream', module: 'stax-api'
         exclude group: 'stax', module: 'stax-api'
+        exclude group: 'stax', module: 'stax'
     }
     
     modules {
diff --git a/buildSrc/src/main/groovy/org/apache/lucene/gradle/JarChecksum.groovy b/buildSrc/src/main/groovy/org/apache/lucene/gradle/JarChecksum.groovy
index 1095797..14bacd8 100644
--- a/buildSrc/src/main/groovy/org/apache/lucene/gradle/JarChecksum.groovy
+++ b/buildSrc/src/main/groovy/org/apache/lucene/gradle/JarChecksum.groovy
@@ -72,7 +72,7 @@ class JarChecksum extends DefaultTask {
         }
       }
       
-      ant.fixcrlf(srcdir: target.getAbsolutePath(), includes: "**/*.jar.sha1", eol: "lf", fixlast: "true", encoding: "US-ASCII")
+      ant.fixcrlf(srcdir: target.getAbsolutePath(), includes: "**/*.jar.sha1", eol: "lf", fixlast: "true", encoding: "UTF-8")
       
     })
     
diff --git a/buildSrc/src/main/groovy/org/apache/lucene/gradle/UnusedDeps.groovy b/buildSrc/src/main/groovy/org/apache/lucene/gradle/UnusedDeps.groovy
new file mode 100644
index 0000000..9a9c123
--- /dev/null
+++ b/buildSrc/src/main/groovy/org/apache/lucene/gradle/UnusedDeps.groovy
@@ -0,0 +1,138 @@
+package org.apache.lucene.gradle
+/*
+ * 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.gradle.api.artifacts.ResolvedArtifact
+
+import javax.inject.Inject
+import org.gradle.api.DefaultTask
+import org.gradle.api.Project
+import org.gradle.api.artifacts.Configuration
+import org.gradle.api.file.RelativePath
+import org.gradle.api.tasks.Input
+import org.gradle.api.tasks.Optional
+import org.gradle.api.tasks.InputDirectory
+import org.gradle.api.tasks.InputFile
+import org.gradle.api.tasks.OutputDirectory
+import org.gradle.api.tasks.TaskAction
+
+import java.nio.file.Files
+import java.util.regex.Matcher
+import java.util.regex.Pattern
+
+// dev util task to help find possible unused deps
+class UnusedDeps extends DefaultTask {
+  protected static Pattern pattern = Pattern.compile("\\((.*?\\.jar)\\)")
+  
+  @Inject
+  public UnusedDeps() {
+
+  }
+  
+  @TaskAction
+  void execute() {
+    // make sure ant task logging shows up by default
+    ant.lifecycleLogLevel = "INFO"
+    String configuration = "runtimeClasspath"
+    
+    if (project.hasProperty('unusedDepsConfig')) {
+      configuration = project.unusedDepsConfig
+    }
+    
+    File tmpDir = File.createTempDir()
+    tmpDir.deleteOnExit()
+    tmpDir.mkdirs()
+    
+    File distDir = new File(tmpDir, 'distDir')
+    distDir.mkdirs()
+    File jdepsDir = new File(tmpDir, 'jdepsDir')
+    jdepsDir.mkdirs()
+    
+
+      Project topLvlProject
+      File destinationDir
+      
+      if (project.group ==~ /.*?\.lucene(?:\.\w+)?/) {
+        topLvlProject = project.project(":lucene")
+        destinationDir = new File(topLvlProject.projectDir, "dist")
+      } else if (project.group ==~ /.*?\.solr(?:\.\w+)?/) {
+        topLvlProject = project.project(":solr")
+        destinationDir = new File(topLvlProject.projectDir, "dist")
+      }
+      
+     
+      ant.untar(src: "${destinationDir}/${topLvlProject.name}-${topLvlProject.version}.tgz", dest: distDir.getAbsolutePath(), compression:"gzip")
+
+      def distPath = "${distDir}/" + topLvlProject.relativePath(project.projectDir)
+      
+      ant.exec (executable: "jdeps", failonerror: true, resolveexecutable: true) {
+        ant.arg(line: '--class-path ' + "${distPath}/lib/" + '*')
+        ant.arg(line: '--multi-release 11')
+        ant.arg(value: '-recursive')
+        ant.arg(value: '-verbose:class')
+        ant.arg(line: "-dotoutput ${jdepsDir.getAbsolutePath()}")
+        ant.arg(value: "${distPath}/${project.name}-${topLvlProject.version}.jar")
+      }
+      
+      
+      Set<String> usedDepJarNames = new HashSet<>()
+      File file = new File(jdepsDir, "${project.name}-${topLvlProject.version}.jar.dot")
+      def lines = file.readLines()
+      
+      lines.each { String line ->
+        Matcher m = pattern.matcher(line)
+        if (m.find()) {
+          usedDepJarNames.add(m.group(1))
+        }
+      }
+      
+      Set<File> ourDeps = new HashSet<>()
+      
+      Configuration config = project.configurations[configuration]
+      if (config.isCanBeResolved()) {
+        config.getResolvedConfiguration().getResolvedArtifacts().forEach( { ra -> ourDeps.add(ra.getFile()) })
+        // exclude our jar and jarTest outputs
+        def ourJars = new ArrayList()
+        project.rootProject.subprojects.each{ subproject ->
+          if (subproject.hasProperty('jar')) {
+            ourJars.addAll(subproject.jar.outputs.files)
+            if (subproject.hasProperty('jarTest')) {
+              ourJars.addAll(subproject.jarTest.outputs.files)
+            }
+          }
+        }
+        ourDeps = ourDeps - ourJars
+      }
+      
+      Set<String> ourDepJarNames = new HashSet<>()
+      ourDeps.forEach( { ourDepJarNames.add(it.getName()) } )
+      
+      println 'our dep count ' + ourDepJarNames.size()
+      println 'used count ' + usedDepJarNames.size()
+      
+      Set<String> unusedJarNames = new HashSet<>()
+      unusedJarNames.addAll(ourDepJarNames)
+      unusedJarNames -= usedDepJarNames
+      unusedJarNames = unusedJarNames.toSorted()
+      unusedJarNames.forEach( { println it } )
+      
+      project.delete(tmpDir)
+
+  }
+}
+
+
diff --git a/lucene/analysis/common/build.gradle b/lucene/analysis/common/build.gradle
index 046d72c..2f18753 100644
--- a/lucene/analysis/common/build.gradle
+++ b/lucene/analysis/common/build.gradle
@@ -19,7 +19,7 @@ apply plugin: 'java-library'
 apply plugin: 'maven-publish'
 apply plugin: org.apache.lucene.gradle.PartOfDist
 
-archivesBaseName = 'analyzers-common'
+archivesBaseName = 'lucene-analyzers-common'
 
 def unicodePropsJavaFile = "/src/java/org/apache/lucene/analysis/util/UnicodeProps.java"
 
diff --git a/lucene/analysis/icu/build.gradle b/lucene/analysis/icu/build.gradle
index 86770c5..a0de8b7 100644
--- a/lucene/analysis/icu/build.gradle
+++ b/lucene/analysis/icu/build.gradle
@@ -19,7 +19,7 @@ apply plugin: 'java-library'
 apply plugin: 'maven-publish'
 apply plugin: org.apache.lucene.gradle.PartOfDist
 
-archivesBaseName = 'analyzers-icu'
+archivesBaseName = 'lucene-analyzers-icu'
 
 configurations {
   tools
diff --git a/lucene/analysis/kuromoji/build.gradle b/lucene/analysis/kuromoji/build.gradle
index b1ef381..5580098 100644
--- a/lucene/analysis/kuromoji/build.gradle
+++ b/lucene/analysis/kuromoji/build.gradle
@@ -19,7 +19,7 @@ apply plugin: 'java-library'
 apply plugin: 'maven-publish'
 apply plugin: org.apache.lucene.gradle.PartOfDist
 
-archivesBaseName = 'analyzers-kuromoji'
+archivesBaseName = 'lucene-analyzers-kuromoji'
 
 configurations {
   tools
diff --git a/lucene/analysis/morfologik/build.gradle b/lucene/analysis/morfologik/build.gradle
index 1021fe9..d959862 100644
--- a/lucene/analysis/morfologik/build.gradle
+++ b/lucene/analysis/morfologik/build.gradle
@@ -19,7 +19,7 @@ apply plugin: 'java-library'
 apply plugin: 'maven-publish'
 apply plugin: org.apache.lucene.gradle.PartOfDist
 
-archivesBaseName = 'analyzers-morfologik'
+archivesBaseName = 'lucene-analyzers-morfologik'
 
 dependencies {
   
diff --git a/lucene/analysis/nori/build.gradle b/lucene/analysis/nori/build.gradle
index 32581d9..2550ea2 100644
--- a/lucene/analysis/nori/build.gradle
+++ b/lucene/analysis/nori/build.gradle
@@ -19,7 +19,7 @@ apply plugin: 'java-library'
 apply plugin: 'maven-publish'
 apply plugin: org.apache.lucene.gradle.PartOfDist
 
-archivesBaseName = 'analyzers-nori'
+archivesBaseName = 'lucene-analyzers-nori'
 
 configurations {
   tools
diff --git a/lucene/analysis/opennlp/build.gradle b/lucene/analysis/opennlp/build.gradle
index 0ff8d29..9d709d4 100644
--- a/lucene/analysis/opennlp/build.gradle
+++ b/lucene/analysis/opennlp/build.gradle
@@ -19,6 +19,8 @@ apply plugin: 'java-library'
 apply plugin: 'maven-publish'
 apply plugin: org.apache.lucene.gradle.PartOfDist
 
+archivesBaseName = 'lucene-analyzers-opennlp'
+
 dependencies {
   implementation project(':lucene:lucene-core')
   implementation project(':lucene:analysis:lucene-analyzers-common')
diff --git a/lucene/analysis/phonetic/build.gradle b/lucene/analysis/phonetic/build.gradle
index 2fe423b..85683e0 100644
--- a/lucene/analysis/phonetic/build.gradle
+++ b/lucene/analysis/phonetic/build.gradle
@@ -19,7 +19,7 @@ apply plugin: 'java-library'
 apply plugin: 'maven-publish'
 apply plugin: org.apache.lucene.gradle.PartOfDist
 
-archivesBaseName = 'analyzers-phonetic'
+archivesBaseName = 'lucene-analyzers-phonetic'
 
 dependencies {
   
diff --git a/lucene/analysis/smartcn/build.gradle b/lucene/analysis/smartcn/build.gradle
index 958d5ed..796f183 100644
--- a/lucene/analysis/smartcn/build.gradle
+++ b/lucene/analysis/smartcn/build.gradle
@@ -19,7 +19,7 @@ apply plugin: 'java-library'
 apply plugin: 'maven-publish'
 apply plugin: org.apache.lucene.gradle.PartOfDist
 
-archivesBaseName = 'analyzers-smartcn'
+archivesBaseName = 'lucene-analyzers-smartcn'
 
 dependencies {
   
diff --git a/lucene/analysis/stempel/build.gradle b/lucene/analysis/stempel/build.gradle
index b52eb4b..b0ad413 100644
--- a/lucene/analysis/stempel/build.gradle
+++ b/lucene/analysis/stempel/build.gradle
@@ -19,7 +19,7 @@ apply plugin: 'java-library'
 apply plugin: 'maven-publish'
 apply plugin: org.apache.lucene.gradle.PartOfDist
 
-archivesBaseName = 'analyzers-stemple'
+archivesBaseName = 'lucene-analyzers-stemple'
 
 dependencies {
   
diff --git a/lucene/build.gradle b/lucene/build.gradle
index 712389e..cb8d785 100644
--- a/lucene/build.gradle
+++ b/lucene/build.gradle
@@ -37,4 +37,9 @@ task packageDist(type: org.apache.lucene.gradle.PackageLuceneSolrDist) {
 task runJflex() {
   group = 'Build Regenerate'
   description = "Runs all jflex targets to regenerate src files."
-}
\ No newline at end of file
+}
+
+subprojects {
+  tasks.create("unusedDeps", org.apache.lucene.gradle.UnusedDeps)
+  unusedDeps.dependsOn packageDist
+}
diff --git a/lucene/expressions/build.gradle b/lucene/expressions/build.gradle
index 701a5f4..628d7df 100644
--- a/lucene/expressions/build.gradle
+++ b/lucene/expressions/build.gradle
@@ -26,6 +26,5 @@ dependencies {
   testImplementation project(':lucene:lucene-test-framework')
   
   implementation 'org.antlr:antlr4-runtime'
-  implementation 'org.ow2.asm:asm'
   implementation 'org.ow2.asm:asm-commons'
 }
diff --git a/solr/build.gradle b/solr/build.gradle
index 2f06581..87f957c 100644
--- a/solr/build.gradle
+++ b/solr/build.gradle
@@ -33,3 +33,8 @@ task packageDist(type: org.apache.lucene.gradle.PackageLuceneSolrDist) {
   group = 'Dist'
   description = "Generates the Lucene and Solr distribution."
 }
+
+subprojects {
+  tasks.create("unusedDeps", org.apache.lucene.gradle.UnusedDeps)
+  unusedDeps.dependsOn packageDist
+}
diff --git a/solr/contrib/analytics/build.gradle b/solr/contrib/analytics/build.gradle
index ba1e425..d4e8d5d 100644
--- a/solr/contrib/analytics/build.gradle
+++ b/solr/contrib/analytics/build.gradle
@@ -28,7 +28,6 @@ dependencies {
   implementation ('com.fasterxml.jackson.core:jackson-databind') { transitive = false }
   implementation ('com.fasterxml.jackson.core:jackson-annotations') { transitive = false }
   implementation ('com.google.guava:guava') { transitive = false }
-  implementation ('org.noggit:noggit') { transitive = false }
   implementation ('commons-codec:commons-codec') { transitive = false }
   implementation ('commons-io:commons-io') { transitive = false }
   implementation ('org.apache.commons:commons-lang3') { transitive = false }
@@ -39,8 +38,6 @@ dependencies {
   // if we try to make this compileOnly or testImplementation forbidden apis fails?
   implementation ('javax.servlet:javax.servlet-api') { transitive = false }
   
-  testRuntimeOnly ('org.noggit:noggit') { transitive = false }
-  
   testRuntimeOnly ('javax.servlet:javax.servlet-api') { transitive = false }
   testRuntimeOnly ('commons-fileupload:commons-fileupload') { transitive = false }
   testRuntimeOnly ('org.apache.httpcomponents:httpclient') { transitive = false }
diff --git a/solr/contrib/clustering/build.gradle b/solr/contrib/clustering/build.gradle
index f14861c..217133f 100644
--- a/solr/contrib/clustering/build.gradle
+++ b/solr/contrib/clustering/build.gradle
@@ -27,17 +27,14 @@ dependencies {
   implementation project(':lucene:lucene-memory')
   implementation project(':lucene:analysis:lucene-analyzers-common')
   
-  implementation ('org.slf4j:slf4j-api') { transitive = false }
-  implementation ('org.carrot2:carrot2-mini') { transitive = false }
-  implementation ('org.carrot2.shaded:carrot2-guava') { transitive = false }
-  implementation ('org.carrot2.attributes:attributes-binder') { transitive = false }
-  implementation ('org.simpleframework:simple-xml') { transitive = false }
-  implementation ('com.fasterxml.jackson.core:jackson-annotations') { transitive = false }
-  implementation ('com.fasterxml.jackson.core:jackson-databind') { transitive = false }
-  implementation ('commons-io:commons-io') { transitive = false }
-  implementation ('com.google.guava:guava') { transitive = false }
-  implementation ('commons-codec:commons-codec') { transitive = false }
-  implementation ('org.apache.commons:commons-lang3') { transitive = false }
+  implementation ('org.slf4j:slf4j-api')
+  implementation ('org.carrot2:carrot2-mini')
+  implementation ('org.carrot2.shaded:carrot2-guava')
+  implementation ('org.carrot2.attributes:attributes-binder')
+  implementation ('org.simpleframework:simple-xml')
+  implementation ('com.fasterxml.jackson.core:jackson-annotations')
+  implementation ('com.fasterxml.jackson.core:jackson-databind')
+  implementation ('commons-io:commons-io')
   
   testImplementation project(':lucene:lucene-test-framework')
   testImplementation project(':solr:solr-test-framework')
diff --git a/solr/contrib/ltr/build.gradle b/solr/contrib/ltr/build.gradle
index bb75ece..67359ba 100644
--- a/solr/contrib/ltr/build.gradle
+++ b/solr/contrib/ltr/build.gradle
@@ -28,7 +28,6 @@ dependencies {
   
   implementation ('org.slf4j:slf4j-api') { transitive = false }
   implementation ('org.restlet.jee:org.restlet') { transitive = false }
-  implementation ('org.noggit:noggit') { transitive = false }
   implementation ('commons-io:commons-io') { transitive = false }
   implementation ('com.google.guava:guava') { transitive = false }
   implementation ('commons-codec:commons-codec') { transitive = false }
diff --git a/solr/contrib/prometheus-exporter/build.gradle b/solr/contrib/prometheus-exporter/build.gradle
index 43ce442..7d2fad3 100644
--- a/solr/contrib/prometheus-exporter/build.gradle
+++ b/solr/contrib/prometheus-exporter/build.gradle
@@ -37,7 +37,6 @@ dependencies {
   implementation ('org.slf4j:slf4j-api') { transitive = false }
   implementation ('com.google.guava:guava') { transitive = false }
   implementation ('org.apache.zookeeper:zookeeper') { transitive = false }
-  implementation ('org.noggit:noggit') { transitive = false }
   implementation ('commons-codec:commons-codec') { transitive = false }
   implementation ('commons-io:commons-io') { transitive = false }
   implementation ('org.apache.httpcomponents:httpclient') { transitive = false }
diff --git a/solr/core/build.gradle b/solr/core/build.gradle
index 7a1760a..a40a87e 100644
--- a/solr/core/build.gradle
+++ b/solr/core/build.gradle
@@ -15,7 +15,11 @@
  * limitations under the License.
  */
 
+ import org.apache.lucene.gradle.UnusedDeps
+ 
 apply plugin: 'java-library'
+apply plugin: 'maven-publish'
+apply plugin: org.apache.lucene.gradle.PartOfDist
 
 archivesBaseName = 'solr-core'
 
@@ -40,6 +44,15 @@ dependencies {
   implementation project(':lucene:lucene-join')
   implementation project(':solr:solr-solrj')
   
+  // kerb, kerby, and curator deps used for security and auth with zk/hdfs/kerberos
+  runtimeOnly ('org.apache.curator:curator-recipes') {
+    exclude group: '*', module: '*' // brings in a lot we don't use
+  }
+  runtimeOnly ('org.apache.kerby:kerb-core')
+  runtimeOnly ('org.apache.kerby:kerb-util')
+  runtimeOnly ('org.apache.kerby:kerby-asn1')
+  runtimeOnly ('org.apache.kerby:kerby-pkix')
+  
   compileOnly ('javax.servlet:javax.servlet-api')
   
   // TODO: this is not great
@@ -51,12 +64,9 @@ dependencies {
     exclude group: '*', module: '*' // zk has many deps we don't need
   }
   implementation ('org.apache.httpcomponents:httpclient')
-  implementation ('org.apache.httpcomponents:httpmime')
   implementation ('org.apache.httpcomponents:httpcore')
   implementation ('org.codehaus.woodstox:stax2-api')
   implementation ('org.codehaus.woodstox:woodstox-core-asl')
-  implementation ('org.ow2.asm:asm')
-  implementation ('org.ow2.asm:asm-commons')
   implementation ('io.dropwizard.metrics:metrics-core') {
     exclude group: 'com.rabbitmq', module: 'amqp-client' // we don't use this
   }
@@ -65,14 +75,13 @@ dependencies {
   }
   implementation ('io.dropwizard.metrics:metrics-jvm')
   implementation ('io.dropwizard.metrics:metrics-jmx')
-  implementation ('org.noggit:noggit')
-  implementation ('commons-codec:commons-codec')
+  api ('commons-codec:commons-codec') // api because of forbidden apis :(
   implementation ('commons-io:commons-io')
   implementation ('org.apache.commons:commons-exec')
   implementation ('commons-fileupload:commons-fileupload')
   implementation ('commons-cli:commons-cli')
   implementation ('org.apache.commons:commons-text')
-  implementation ('com.google.guava:guava') {
+  api ('com.google.guava:guava') {  // api because of forbidden apis :(
     exclude group: '*', module: '*' // many annotation deps we don't need
   }
   implementation ('org.locationtech.spatial4j:spatial4j')
@@ -104,7 +113,6 @@ dependencies {
   implementation ('org.apache.hadoop:hadoop-annotations') {
     exclude group: '*', module: '*' // brings in a lot we don't use
   }
-  implementation ('org.apache.commons:commons-configuration2')
   implementation ('commons-collections:commons-collections')
   implementation ('commons-beanutils:commons-beanutils')
   implementation ('com.github.ben-manes.caffeine:caffeine')
@@ -117,13 +125,6 @@ dependencies {
   implementation ('org.apache.curator:curator-client')  {
     exclude group: '*', module: '*' // brings in a lot we don't use
   }
-  implementation ('org.apache.curator:curator-recipes') {
-    exclude group: '*', module: '*' // brings in a lot we don't use
-  }
-  implementation ('org.apache.kerby:kerb-core')
-  implementation ('org.apache.kerby:kerb-util')
-  implementation ('org.apache.kerby:kerby-asn1')
-  implementation ('org.apache.kerby:kerby-pkix')
   
   implementation ('com.tdunning:t-digest')
   implementation ('org.apache.calcite:calcite-core') {
@@ -136,33 +137,34 @@ dependencies {
     exclude group: '*', module: '*' // brings in a lot we don't use
   }
   implementation ('org.apache.commons:commons-lang3')
-  implementation ('net.hydromatic:eigenbase-properties')
-  implementation ('org.codehaus.janino:janino')
   implementation ('org.codehaus.janino:commons-compiler')
-  implementation ('com.google.protobuf:protobuf-java') 
-  implementation ('com.jayway.jsonpath:json-path')
+
   implementation ('org.rrd4j:rrd4j')
   implementation ('org.bitbucket.b_c:jose4j')
   
   implementation ('org.eclipse.jetty:jetty-alpn-server')
   implementation ('org.eclipse.jetty.http2:http2-server')
-  implementation ('org.eclipse.jetty.http2:http2-client')
-  implementation ('org.eclipse.jetty.http2:http2-http-client-transport')
   implementation ('org.eclipse.jetty.http2:http2-common')
-  implementation ('org.eclipse.jetty.http2:http2-hpack')
   implementation ('org.eclipse.jetty:jetty-server')
   implementation ('org.eclipse.jetty:jetty-client')
   implementation ('org.eclipse.jetty:jetty-util')
-  implementation ('org.eclipse.jetty:jetty-http')
-  implementation ('org.eclipse.jetty:jetty-io')
-  implementation ('org.eclipse.jetty:jetty-xml')
-  implementation ('org.eclipse.jetty:jetty-alpn-java-client')
-  implementation ('org.eclipse.jetty:jetty-alpn-client')
   implementation ('org.eclipse.jetty:jetty-servlet')
   implementation ('org.eclipse.jetty:jetty-server')
   
   testRuntimeOnly ('com.lmax:disruptor')
   
+  // kerby
+  testRuntimeOnly ('org.apache.kerby:kerb-core')
+  testRuntimeOnly ('org.apache.kerby:kerby-asn1')
+  testRuntimeOnly ('org.apache.kerby:kerby-pkix')
+  testRuntimeOnly ('org.apache.kerby:kerby-config')
+  testRuntimeOnly ('org.apache.kerby:kerb-crypto')
+  testRuntimeOnly ('org.apache.kerby:kerb-util')
+  
+  // hdfs
+  testRuntimeOnly ('org.apache.commons:commons-configuration2')
+  
+  testImplementation ('org.eclipse.jetty:jetty-xml')
   testImplementation ('org.mockito:mockito-core') { transitive = false }
   testImplementation ('net.bytebuddy:byte-buddy') { transitive = false }
   testImplementation ('org.objenesis:objenesis') { transitive = false }
@@ -178,16 +180,14 @@ dependencies {
   testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
   testImplementation ('org.apache.hadoop:hadoop-hdfs::tests') { transitive = false }
   testImplementation ('org.apache.hadoop:hadoop-minikdc') { transitive = false }
+  testImplementation ('com.google.protobuf:protobuf-java')
   
   // kerby
-  testImplementation ('org.apache.kerby:kerby-config') { transitive = false }
   testImplementation ('org.apache.kerby:kerb-client') { transitive = false }
   testImplementation ('org.apache.kerby:kerb-common') { transitive = false }
-  testImplementation ('org.apache.kerby:kerb-crypto') { transitive = false }
   testImplementation ('org.apache.kerby:kerb-identity') { transitive = false }
   testImplementation ('org.apache.kerby:kerb-server') { transitive = false }
   testImplementation ('org.apache.kerby:kerb-simplekdc') { transitive = false }
-  testImplementation ('org.apache.kerby:kerby-util') { transitive = false }
   testImplementation ('org.apache.kerby:kerb-admin') { transitive = false }
   testImplementation ('org.apache.kerby:kerby-kdc') { transitive = false }
   
diff --git a/solr/solrj/build.gradle b/solr/solrj/build.gradle
index 703b996..de44e2f 100644
--- a/solr/solrj/build.gradle
+++ b/solr/solrj/build.gradle
@@ -20,9 +20,7 @@ apply plugin: 'maven-publish'
 apply plugin: org.apache.lucene.gradle.PartOfDist
 
 dependencies {
-  
-  runtimeOnly ('org.eclipse.jetty:jetty-security') { transitive = false }
-  
+
   implementation ('org.apache.zookeeper:zookeeper') {
     exclude group: '*', module: '*' // zk has many deps we don't need
   }
@@ -32,16 +30,11 @@ dependencies {
   implementation ('org.slf4j:jcl-over-slf4j')
   implementation ('commons-io:commons-io')
   implementation ('org.apache.commons:commons-math3')
-  implementation ('org.noggit:noggit')
   implementation ('org.slf4j:slf4j-api')
   implementation ('org.eclipse.jetty.http2:http2-client')
   implementation ('org.eclipse.jetty.http2:http2-http-client-transport')
-  implementation ('org.eclipse.jetty.http2:http2-common')
-  implementation ('org.eclipse.jetty.http2:http2-hpack')
-  implementation ('org.eclipse.jetty:jetty-client')
   implementation ('org.eclipse.jetty:jetty-util')
   implementation ('org.eclipse.jetty:jetty-http')
-  implementation ('commons-codec:commons-codec')
   
   testImplementation ('org.eclipse.jetty:jetty-servlet')
   testImplementation ('io.dropwizard.metrics:metrics-core')
@@ -68,3 +61,4 @@ dependencies {
   testImplementation project(':lucene:analysis:lucene-analyzers-common')
   testImplementation project(':solr:example:solr-example-DIH')
 }
+
diff --git a/solr/test-framework/build.gradle b/solr/test-framework/build.gradle
index 53a5dff..47a4e53 100644
--- a/solr/test-framework/build.gradle
+++ b/solr/test-framework/build.gradle
@@ -38,7 +38,6 @@ dependencies {
   implementation ('org.apache.httpcomponents:httpmime') { transitive = false }
   implementation ('org.slf4j:slf4j-api') { transitive = false }
   implementation ('org.apache.logging.log4j:log4j-api') { transitive = false }
-  implementation ('org.noggit:noggit') { transitive = false }
   implementation ('org.apache.zookeeper:zookeeper') { transitive = false }
   implementation ('commons-io:commons-io') { transitive = false }
   api ('org.eclipse.jetty:jetty-util') { transitive = false }
@@ -83,4 +82,5 @@ dependencies {
   runtimeOnly ('org.apache.logging.log4j:log4j-slf4j-impl') { transitive = false }
   runtimeOnly ('com.lmax:disruptor') { transitive = false }
   
-}
\ No newline at end of file
+}
+
diff --git a/versions.lock b/versions.lock
index 33d7ce2..424d337 100644
--- a/versions.lock
+++ b/versions.lock
@@ -2,14 +2,14 @@
 c3p0:c3p0:0.9.1.1 (1 constraints: eb0c6e1a)
 com.adobe.xmp:xmpcore:5.1.3 (2 constraints: cf13ce64)
 com.beust:jcommander:1.35 (1 constraints: f805cf47)
-com.carrotsearch:hppc:0.8.1 (1 constraints: 0b050436)
+com.carrotsearch:hppc:0.8.1 (2 constraints: af0fd8a6)
 com.carrotsearch.randomizedtesting:randomizedtesting-runner:2.7.2 (1 constraints: 0d050c36)
 com.cybozu.labs:langdetect:1.1-20120112 (1 constraints: 5c066d5e)
 com.drewnoakes:metadata-extractor:2.11.0 (2 constraints: c011970f)
 com.epam:parso:2.0.9 (2 constraints: 6e11a9f2)
-com.fasterxml.jackson.core:jackson-annotations:2.9.8 (3 constraints: d62310b2)
-com.fasterxml.jackson.core:jackson-core:2.9.8 (4 constraints: ed3b2294)
-com.fasterxml.jackson.core:jackson-databind:2.9.8 (2 constraints: 7c1103f4)
+com.fasterxml.jackson.core:jackson-annotations:2.9.8 (4 constraints: ad2e114a)
+com.fasterxml.jackson.core:jackson-core:2.9.8 (5 constraints: c4465e5f)
+com.fasterxml.jackson.core:jackson-databind:2.9.8 (3 constraints: b21c2555)
 com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.9.8 (1 constraints: 15051836)
 com.fasterxml.woodstox:woodstox-core:5.2.0 (1 constraints: 360a63bc)
 com.github.ben-manes.caffeine:caffeine:2.4.0 (1 constraints: 08050136)
@@ -31,7 +31,6 @@ com.googlecode.mp4parser:isoparser:1.1.22 (2 constraints: c411d90f)
 com.healthmarketscience.jackcess:jackcess:2.1.12 (2 constraints: c411e10f)
 com.healthmarketscience.jackcess:jackcess-encrypt:2.1.4 (2 constraints: 661111f2)
 com.ibm.icu:icu4j:62.1 (1 constraints: dd040c31)
-com.jayway.jsonpath:json-path:2.4.0 (1 constraints: 08050136)
 com.jcraft:jsch:0.1.53 (1 constraints: 790ef15e)
 com.lmax:disruptor:3.4.0 (1 constraints: 09050636)
 com.pff:java-libpst:0.8.1 (2 constraints: 6a116bf2)
@@ -75,11 +74,8 @@ javax.xml.bind:jaxb-api:2.3.0 (1 constraints: 5f0c0706)
 joda-time:joda-time:2.2 (2 constraints: 5d0d232c)
 junit:junit:4.12 (2 constraints: 3e1e6104)
 net.arnx:jsonic:1.2.7 (2 constraints: db10d4d1)
-net.hydromatic:eigenbase-properties:1.1.5 (1 constraints: 0905f835)
 net.java.dev.jna:jna:4.3.0 (1 constraints: 5e0cf901)
 net.jcip:jcip-annotations:1.0 (5 constraints: 6824b237)
-net.minidev:accessors-smart:1.2 (1 constraints: e60926a4)
-net.minidev:json-smart:2.3 (1 constraints: a40c49ff)
 net.sf.ehcache:ehcache-core:2.6.2 (1 constraints: 2706f94d)
 net.sourceforge.argparse4j:argparse4j:0.8.1 (1 constraints: 0b050436)
 net.sourceforge.nekohtml:nekohtml:1.9.17 (1 constraints: 4405503b)
@@ -90,10 +86,9 @@ org.apache.calcite:calcite-linq4j:1.18.0 (1 constraints: 3c05413b)
 org.apache.calcite.avatica:avatica-core:1.13.0 (1 constraints: 3705323b)
 org.apache.commons:commons-collections4:4.2 (2 constraints: b90cdc15)
 org.apache.commons:commons-compress:1.18 (4 constraints: 2128b5ae)
-org.apache.commons:commons-configuration2:2.1.1 (1 constraints: 0605f935)
 org.apache.commons:commons-csv:1.5 (1 constraints: ff0b57e9)
 org.apache.commons:commons-exec:1.3 (2 constraints: a41056b8)
-org.apache.commons:commons-lang3:3.8.1 (3 constraints: 2824aaaf)
+org.apache.commons:commons-lang3:3.8.1 (3 constraints: bf1cfe0d)
 org.apache.commons:commons-math3:3.6.1 (1 constraints: 0c050d36)
 org.apache.commons:commons-text:1.6 (1 constraints: ab04202c)
 org.apache.curator:curator-client:2.13.0 (1 constraints: 3805383b)
@@ -167,18 +162,16 @@ org.carrot2:carrot2-mini:3.16.0 (1 constraints: 3c05473b)
 org.carrot2:morfologik-fsa:2.1.5 (1 constraints: 0a05fd35)
 org.carrot2:morfologik-polish:2.1.5 (1 constraints: 0a05fd35)
 org.carrot2:morfologik-stemming:2.1.5 (1 constraints: 0a05fd35)
-org.carrot2.attributes:attributes-binder:1.3.3 (1 constraints: 0905fc35)
-org.carrot2.shaded:carrot2-guava:18.0 (1 constraints: dd040931)
+org.carrot2.attributes:attributes-binder:1.3.3 (2 constraints: ab0f86a6)
+org.carrot2.shaded:carrot2-guava:18.0 (3 constraints: 8f202231)
 org.ccil.cowan.tagsoup:tagsoup:1.2.1 (2 constraints: 601198f1)
 org.checkerframework:checker-qual:2.0.0 (1 constraints: 140ae5b4)
-org.codehaus.janino:commons-compiler:3.0.9 (2 constraints: d910f7d1)
-org.codehaus.janino:janino:3.0.9 (1 constraints: 0e050336)
+org.codehaus.janino:commons-compiler:3.0.9 (1 constraints: 0e050336)
 org.codehaus.mojo:animal-sniffer-annotations:1.14 (1 constraints: ea09d5aa)
 org.codehaus.woodstox:stax2-api:4.1 (3 constraints: f125f836)
 org.codehaus.woodstox:woodstox-core-asl:4.4.1 (1 constraints: 0b050c36)
 org.codelibs:jhighlight:1.0.3 (1 constraints: 5b0ce401)
-org.eclipse.jetty:jetty-alpn-client:9.4.14.v20181114 (3 constraints: bf2ca4fb)
-org.eclipse.jetty:jetty-alpn-java-client:9.4.14.v20181114 (1 constraints: 7a070e7d)
+org.eclipse.jetty:jetty-alpn-client:9.4.14.v20181114 (1 constraints: c1111a3a)
 org.eclipse.jetty:jetty-alpn-server:9.4.14.v20181114 (1 constraints: 7a070e7d)
 org.eclipse.jetty:jetty-client:9.4.14.v20181114 (2 constraints: 411fc83d)
 org.eclipse.jetty:jetty-continuation:9.4.14.v20181114 (1 constraints: 7a070e7d)
@@ -191,12 +184,12 @@ org.eclipse.jetty:jetty-security:9.4.14.v20181114 (2 constraints: de1772dc)
 org.eclipse.jetty:jetty-server:9.4.14.v20181114 (4 constraints: f73bfbd7)
 org.eclipse.jetty:jetty-servlet:9.4.14.v20181114 (1 constraints: 7a070e7d)
 org.eclipse.jetty:jetty-servlets:9.4.14.v20181114 (1 constraints: 7a070e7d)
-org.eclipse.jetty:jetty-util:9.4.14.v20181114 (5 constraints: c8449c9d)
+org.eclipse.jetty:jetty-util:9.4.14.v20181114 (4 constraints: 1836c721)
 org.eclipse.jetty:jetty-webapp:9.4.14.v20181114 (1 constraints: 7a070e7d)
 org.eclipse.jetty:jetty-xml:9.4.14.v20181114 (1 constraints: 7a070e7d)
 org.eclipse.jetty.http2:http2-client:9.4.14.v20181114 (2 constraints: 411fc83d)
 org.eclipse.jetty.http2:http2-common:9.4.14.v20181114 (3 constraints: 122b0a1b)
-org.eclipse.jetty.http2:http2-hpack:9.4.14.v20181114 (2 constraints: 4419b95a)
+org.eclipse.jetty.http2:http2-hpack:9.4.14.v20181114 (1 constraints: cb11153b)
 org.eclipse.jetty.http2:http2-http-client-transport:9.4.14.v20181114 (1 constraints: 7a070e7d)
 org.eclipse.jetty.http2:http2-server:9.4.14.v20181114 (1 constraints: 7a070e7d)
 org.eclipse.jgit:org.eclipse.jgit:4.6.0.201612231935-r (1 constraints: 3c085c99)
@@ -212,9 +205,8 @@ org.jdom:jdom2:2.0.6 (2 constraints: 681134f2)
 org.jsoup:jsoup:1.11.3 (2 constraints: c411dd0f)
 org.jvnet.staxex:stax-ex:1.7.8 (1 constraints: c50dc135)
 org.locationtech.spatial4j:spatial4j:0.7 (1 constraints: ab041e2c)
-org.noggit:noggit:0.8 (1 constraints: ac041f2c)
 org.opengis:geoapi:3.0.1 (7 constraints: 1361d046)
-org.ow2.asm:asm:6.2 (5 constraints: 772fb0b6)
+org.ow2.asm:asm:6.2 (4 constraints: 2223c0a3)
 org.ow2.asm:asm-analysis:6.2 (1 constraints: e309d6a5)
 org.ow2.asm:asm-commons:6.2 (1 constraints: aa04272c)
 org.ow2.asm:asm-tree:6.2 (2 constraints: 2d14228c)
@@ -222,11 +214,12 @@ org.quartz-scheduler:quartz:2.2.0 (1 constraints: 2106eb4d)
 org.restlet.jee:org.restlet:2.3.0 (2 constraints: e3159ee6)
 org.restlet.jee:org.restlet.ext.servlet:2.3.0 (1 constraints: 0705fe35)
 org.rrd4j:rrd4j:3.5 (1 constraints: ac04252c)
-org.simpleframework:simple-xml:2.7.1 (1 constraints: 0c050b36)
-org.slf4j:jcl-over-slf4j:1.7.25 (6 constraints: 974ad8ec)
+org.simpleframework:simple-xml:2.7.1 (3 constraints: bd20563c)
+org.slf4j:jcl-over-slf4j:1.7.25 (5 constraints: 4c39d741)
 org.slf4j:jul-to-slf4j:1.7.25 (2 constraints: d5113e11)
-org.slf4j:slf4j-api:1.7.25 (23 constraints: 0c05f93d)
+org.slf4j:slf4j-api:1.7.25 (23 constraints: a802e9ff)
 org.tallison:jmatio:1.5 (2 constraints: a810a0b8)
 org.tukaani:xz:1.8 (2 constraints: ae100fb9)
 ua.net.nlp:morfologik-ukrainian-search:3.9.0 (1 constraints: 0e051536)
 xerces:xercesImpl:2.9.1 (1 constraints: 0e051136)
+xpp3:xpp3:1.1.3.3 (1 constraints: 0e0ef84b)