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/07 08:04:42 UTC

[lucene-solr] 01/05: SOLR-13452: Improve unused dep checker: break it up into jdep and unused tasks, output improvements, etc.

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 a94e430ca551ae67b6b8f88880c9de3104c02941
Author: markrmiller <ma...@apache.org>
AuthorDate: Thu Jun 6 11:53:11 2019 -0500

    SOLR-13452: Improve unused dep checker: break it up into jdep and unused tasks, output improvements, etc.
---
 .../org/apache/lucene/gradle/JdepsReport.groovy    | 166 +++++++++++++++++++
 .../org/apache/lucene/gradle/UnusedDeps.groovy     | 177 +++++++++------------
 lucene/build.gradle                                |   6 +-
 lucene/test-framework/build.gradle                 |   3 +-
 solr/build.gradle                                  |   5 +-
 solr/contrib/clustering/build.gradle               |   2 -
 solr/contrib/dataimporthandler/build.gradle        |  12 +-
 solr/core/build.gradle                             |   2 +-
 8 files changed, 254 insertions(+), 119 deletions(-)

diff --git a/buildSrc/src/main/groovy/org/apache/lucene/gradle/JdepsReport.groovy b/buildSrc/src/main/groovy/org/apache/lucene/gradle/JdepsReport.groovy
new file mode 100644
index 0000000..5c48e2b
--- /dev/null
+++ b/buildSrc/src/main/groovy/org/apache/lucene/gradle/JdepsReport.groovy
@@ -0,0 +1,166 @@
+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 org.gradle.api.artifacts.ResolvedDependency
+import org.gradle.api.artifacts.result.DependencyResult
+import org.gradle.api.artifacts.result.ResolvedDependencyResult
+
+import javax.inject.Inject
+
+import org.apache.tools.ant.types.resources.selectors.InstanceOf
+import org.gradle.api.DefaultTask
+import org.gradle.api.Project
+import org.gradle.api.artifacts.Configuration
+import org.gradle.api.artifacts.Dependency
+import org.gradle.api.artifacts.ModuleIdentifier
+import org.gradle.api.file.RelativePath
+import org.gradle.api.internal.artifacts.dependencies.DefaultProjectDependency
+import org.gradle.api.specs.Spec
+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
+
+class JdepsReport extends DefaultTask {
+  
+  protected configuration = "runtimeClasspath"
+
+  protected File distDir
+  protected File jdepsDir
+  
+  @OutputDirectory
+  File target
+
+  @Inject
+  public JdepsReport(File target) {
+    if (!project.configurations.hasProperty('runtimeClasspath')) {
+      return
+    }
+
+    this.target = target
+    
+    doFirst {
+      println "Writing output files to ${target}"
+    }
+    
+    if (project.hasProperty('unusedDepsConfig')) {
+      configuration = project.unusedDepsConfig
+    }
+    
+    Configuration config = project.configurations[this.configuration]
+    
+    List<Project> buildProjects = new ArrayList()
+    buildProjects.add(project)
+    config.getAllDependencies().forEach({ dep ->
+      if (dep instanceof DefaultProjectDependency) {
+        Project dProject = dep.getDependencyProject()
+        buildProjects.add(dProject)
+      }
+    })
+    
+    project.tasks.create(name: "depsToDir", type: org.gradle.api.tasks.Copy) {
+      outputs.upToDateWhen { false }
+      into({ makeDirs(); distDir })
+      buildProjects.each {subproject ->
+        project.evaluationDependsOn(subproject.path)
+        def topLvlProject = getTopLvlProject(subproject)
+        
+        if (subproject.getPlugins().hasPlugin(PartOfDist) && subproject.tasks.findByName('jar') && subproject.configurations.hasProperty('runtimeClasspath')) {
+           from(subproject.jar.outputs.files) {
+            include "*.jar"
+            into ({topLvlProject.name + '/' + topLvlProject.relativePath(subproject.projectDir)})
+          }
+          def files = { getFiles(subproject) }
+          from(files) {
+            include "*.jar"
+            into ({topLvlProject.name + '/' + topLvlProject.relativePath(subproject.projectDir) + "/lib"})
+          }
+        }
+      }
+      
+      includeEmptyDirs = false
+    }
+    
+    dependsOn project.tasks.depsToDir
+  }
+  
+  protected void makeDirs() {
+    target.mkdirs()
+    distDir = new File(target, 'distDir')
+    jdepsDir = new File(target, 'jdepsDir')
+    distDir.mkdirs()
+    jdepsDir.mkdirs()
+  }
+
+  
+  @TaskAction
+  void execute() {
+    // make sure ant task logging shows up by default
+    ant.lifecycleLogLevel = "INFO"
+
+    runJdeps(getTopLvlProject(project), project, distDir, jdepsDir)
+    
+    Configuration config = project.configurations[this.configuration]
+    config.getAllDependencies().forEach({ dep ->
+      if (dep instanceof DefaultProjectDependency) {
+        Project dProject = dep.getDependencyProject()
+        def depTopLvlProject = getTopLvlProject(dProject)
+        
+        runJdeps(depTopLvlProject, dProject, distDir, jdepsDir)
+      }
+    })
+  }
+  
+  protected void runJdeps(Project topLvlProject, Project project, File distDir, File jdepsDir) {
+    def distPath = "${distDir}/" + topLvlProject.name + "/" + topLvlProject.relativePath(project.projectDir)
+    def dotOutPath = jdepsDir.getAbsolutePath() + "/" + topLvlProject.name +  "/" + "${project.name}-${project.version}"
+    
+    ant.exec (executable: "jdeps", failonerror: true, resolveexecutable: true) {
+      ant.arg(line: '--class-path ' + "${distPath}/lib/" + '*')
+      ant.arg(line: '--multi-release 11')
+      ant.arg(value: '-verbose:class')
+      ant.arg(line: "-dotoutput ${dotOutPath}")
+      ant.arg(value: "${distPath}/${project.name}-${project.version}.jar")
+    }
+  }
+  
+  private static Collection getFiles(Project subproject) {
+    def files = subproject.configurations.runtimeClasspath.files
+    
+    return files
+  }
+  
+  protected Project getTopLvlProject(Project proj) {
+    def topLvlProject
+    if (proj.group ==~ /.*?\.lucene(?:\.\w+)?/) {
+      topLvlProject = project.project(":lucene")
+    } else if (proj.group ==~ /.*?\.solr(?:\.\w+)?/) {
+      topLvlProject = project.project(":solr")
+    }
+    return topLvlProject
+  }
+}
+
+
diff --git a/buildSrc/src/main/groovy/org/apache/lucene/gradle/UnusedDeps.groovy b/buildSrc/src/main/groovy/org/apache/lucene/gradle/UnusedDeps.groovy
index aed0703..fe6fd53 100644
--- a/buildSrc/src/main/groovy/org/apache/lucene/gradle/UnusedDeps.groovy
+++ b/buildSrc/src/main/groovy/org/apache/lucene/gradle/UnusedDeps.groovy
@@ -17,13 +17,15 @@ package org.apache.lucene.gradle
  */
 
 import org.gradle.api.artifacts.ResolvedArtifact
-
+import org.gradle.api.artifacts.ResolvedDependency
 import javax.inject.Inject
 import org.gradle.api.DefaultTask
 import org.gradle.api.Project
 import org.gradle.api.artifacts.Configuration
+import org.gradle.api.artifacts.Dependency
 import org.gradle.api.file.RelativePath
 import org.gradle.api.internal.artifacts.dependencies.DefaultProjectDependency
+import org.gradle.api.specs.Spec
 import org.gradle.api.tasks.Input
 import org.gradle.api.tasks.Optional
 import org.gradle.api.tasks.InputDirectory
@@ -40,16 +42,24 @@ class UnusedDeps extends DefaultTask {
   protected static Pattern pattern = Pattern.compile("\\(([^\\s]*?\\.jar)\\)")
   
   protected configuration = "runtimeClasspath"
-  protected File tmpDir
   protected File distDir
   protected File jdepsDir
   
-  public UnusedDeps() {
-    //!project.getPlugins().hasPlugin('java-base') ||
+  @InputDirectory
+  File inputDirectory
+  
+  @Inject
+  public UnusedDeps(File inputDirectory) {
+    
     if (!project.configurations.hasProperty('runtimeClasspath')) {
       return
     }
     
+    this.inputDirectory = inputDirectory
+    
+    distDir = new File(inputDirectory, 'distDir')
+    jdepsDir = new File(inputDirectory, 'jdepsDir')
+    
     if (project.hasProperty('unusedDepsConfig')) {
       configuration = project.unusedDepsConfig
     }
@@ -64,61 +74,6 @@ class UnusedDeps extends DefaultTask {
         buildProjects.add(dProject)
       }
     })
-    
-    project.tasks.create(name: "export", type: org.gradle.api.tasks.Copy) {
-      outputs.upToDateWhen { false }
-      into({ makeDirs(); distDir })
-      buildProjects.each {subproject ->
-        project.evaluationDependsOn(subproject.path)
-        def topLvlProject
-        if (subproject.group ==~ /.*?\.lucene(?:\.\w+)?/) {
-          topLvlProject = project.project(":lucene")
-        } else if (subproject.group ==~ /.*?\.solr(?:\.\w+)?/) {
-          topLvlProject = project.project(":solr")
-        }
-        
-        if (subproject.getPlugins().hasPlugin(PartOfDist) && subproject.tasks.findByName('jar') && subproject.configurations.hasProperty('runtimeClasspath')) {
-          from(subproject.jar.outputs.files) {
-            include "*.jar"
-            into ({topLvlProject.name + '/' + topLvlProject.relativePath(subproject.projectDir)})
-          }
-          def files = { getFiles(subproject) }
-          from(files) {
-            include "*.jar"
-            into ({topLvlProject.name + '/' + topLvlProject.relativePath(subproject.projectDir) + "/lib"})
-          }
-        }
-      }
-      
-      includeEmptyDirs = false
-    }
-    
-    dependsOn project.tasks.export
-  }
-  
-  protected void makeDirs() {
-    tmpDir = File.createTempDir()
-    tmpDir.deleteOnExit()
-    tmpDir.mkdirs()
-    distDir = new File(tmpDir, 'distDir')
-    jdepsDir = new File(tmpDir, 'jdepsDir')
-    distDir.mkdirs()
-    jdepsDir.mkdirs()
-  }
-  
-  private static Collection getFiles(Project subproject) {
-    def files = subproject.configurations.runtimeClasspath.files
-    if (!subproject.name.equals('solr-core') && subproject.path.startsWith(":solr:contrib:")) {
-      subproject.evaluationDependsOn(subproject.rootProject.project(":solr:solr-core").path)
-      files = files - subproject.rootProject.project(":solr:solr-core").configurations.runtimeClasspath.files
-      files = files - subproject.rootProject.project(":solr:solr-core").jar.outputs.files
-    }
-    if (!subproject.name.equals('lucene-core') && subproject.path.startsWith(":lucene:")) {
-      subproject.evaluationDependsOn(subproject.rootProject.project(":lucene:lucene-core").path)
-      files = files - subproject.rootProject.project(":lucene:lucene-core").configurations.runtimeClasspath.files
-      files = files - subproject.rootProject.project(":lucene:lucene-core").jar.outputs.files
-    }
-    return files
   }
   
   @TaskAction
@@ -126,40 +81,30 @@ class UnusedDeps extends DefaultTask {
     // make sure ant task logging shows up by default
     ant.lifecycleLogLevel = "INFO"
     
-    Project topLvlProject
-    
-    if (project.group ==~ /.*?\.lucene(?:\.\w+)?/) {
-      topLvlProject = project.project(":lucene")
-    } else if (project.group ==~ /.*?\.solr(?:\.\w+)?/) {
-      topLvlProject = project.project(":solr")
-    }
-    
-    def luceneDist  = new File("/data1/mark/tmp", "lucene")
-    def solrDist = new File("/data1/mark/tmp", "solr")
+    def topLvlProject = getTopLvlProject(project)
     
     Configuration config = project.configurations[this.configuration]
     
-    Set<String> usedDepJarNames = getDirectlyUsedDeps(topLvlProject, project, distDir, jdepsDir)
+    Set<String> usedDepJarNames = getDefinedDeps(topLvlProject, project, distDir, jdepsDir)
+    
+    Set<File> ourDeps = getAllDefinedDeps(project, config)
     
-    Set<File> ourDeps = getDeps(project, config)
+    Set<File> ourImmediatelyDefinedDeps = getOurImmediateDefinedDeps(project, config)
     
     config.getAllDependencies().forEach({ dep ->
       if (dep instanceof DefaultProjectDependency) {
         Project dProject = dep.getDependencyProject()
-        def depTopLvlProject
-        if (dProject.group ==~ /.*?\.lucene(?:\.\w+)?/) {
-          depTopLvlProject = project.project(":lucene")
-        } else if (dProject.group ==~ /.*?\.solr(?:\.\w+)?/) {
-          depTopLvlProject = project.project(":solr")
-        }
+        def depTopLvlDProject = getTopLvlProject(dProject)
         
-        Set<String> projectUsedDeps = getDirectlyUsedDeps(depTopLvlProject, dProject, distDir, jdepsDir)
+        Set<String> projectUsedDeps = getDefinedDeps(depTopLvlDProject, dProject, distDir, jdepsDir)
         
         usedDepJarNames += projectUsedDeps
       }
     })
     
-    usedDepJarNames -= ["${project.name}-${project.version}.jar"]
+    usedDepJarNames -= [
+      "${project.name}-${project.version}.jar"
+    ]
     
     Set<String> ourDepJarNames = new HashSet<>()
     ourDeps.forEach( { ourDepJarNames.add(it.getName()) } )
@@ -169,64 +114,80 @@ class UnusedDeps extends DefaultTask {
     unusedJarNames -= usedDepJarNames
     unusedJarNames = unusedJarNames.toSorted()
     
-    Set<String> foundDeps = new HashSet<>()
+    Set<String> depsInDirectUse = new HashSet<>()
     
     File jdepsLucene = new File(jdepsDir, "lucene")
     File jdepsSolr = new File(jdepsDir, "solr")
     
     for (File file : jdepsLucene.listFiles()) {
-      lookForDep(file, foundDeps)
+      lookForDep(file, depsInDirectUse)
     }
     for (File file : jdepsSolr.listFiles()) {
-      lookForDep(file, foundDeps)
+      lookForDep(file, depsInDirectUse)
     }
     
     println ''
     println 'Our classpath dependency count ' + ourDepJarNames.size()
     println 'Our directly used dependency count ' + usedDepJarNames.size()
     println ''
-    println 'List of possibly unused jars - they may be used at runtime however (Class.forName or something), this is not definitive.'
-    println 'We take our classpath dependenies, substract our direct dependencies and then subtract dependencies used by our direct dependencies'
+    println 'List of possibly unused jars - they may be used at runtime however (Class.forName on plugins or config text for example). This is not definitive, but helps narrow down what to investigate.'
+    println 'We take our classpath dependencies, substract our direct dependencies and then subtract dependencies used by our direct dependencies.'
     println ''
     
+    println 'Direct deps that may be unused:'
     unusedJarNames.forEach({
-      if (!foundDeps.contains(it)) {
-        println it
+      if (!depsInDirectUse.contains(it) && ourImmediatelyDefinedDeps.contains(it)) {
+        println ' - ' + it
       }
     })
     
-    project.delete(tmpDir)
+    println ''
+    println 'Deps brought in by other modules that may be unused in this module:'
+    unusedJarNames.forEach({
+      if (!depsInDirectUse.contains(it) && !ourImmediatelyDefinedDeps.contains(it)) {
+        println ' - ' + it
+      }
+    })
   }
   
-  protected Set getDeps(Project project, Configuration config) {
+  static class NonProjectSpec implements Spec<Dependency> {
+    @Override
+    public boolean isSatisfiedBy(Dependency dep) {
+      return true
+    }
+  }
+  
+  protected Set getAllDefinedDeps(Project project, Configuration config) {
     Set<File> ourDeps = new HashSet<>()
     
     if (config.isCanBeResolved()) {
       config.getResolvedConfiguration().getResolvedArtifacts().forEach( { ra -> ourDeps.add(ra.getFile()) })
     }
+    
     return ourDeps
   }
   
-  protected Set getDirectlyUsedDeps(Project topLvlProject, Project project, File distDir, File jdepsDir) {
-    def distPath = "${distDir}/" + topLvlProject.name + "/" + topLvlProject.relativePath(project.projectDir)
-    def dotOutPath = jdepsDir.getAbsolutePath() + "/" + topLvlProject.name +  "/" + "${project.name}-${project.version}"
+  protected Set getOurImmediateDefinedDeps(Project project, Configuration config) {
+    Set<String> ourDeps = new HashSet<>()
     
-    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 ${dotOutPath}")
-      ant.arg(value: "${distPath}/${project.name}-${project.version}.jar")
+    Set<ResolvedDependency> deps = project.configurations.runtimeClasspath.getResolvedConfiguration().getFirstLevelModuleDependencies(new NonProjectSpec())
+    
+    for (ResolvedDependency dep : deps) {
+      dep.getModuleArtifacts().forEach({ourDeps.add(it.file.name)})
     }
     
+    return ourDeps
+  }
+  
+  protected Set getDefinedDeps(Project topLvlProject, Project project, File distDir, File jdepsDir) {
+    
     File dotFile = new File(jdepsDir, topLvlProject.name +  "/" + "${project.name}-${project.version}/${project.name}-${project.version}.jar.dot")
-    Set<String> usedDepJarNames = getUsedJars(project, dotFile)
+    Set<String> usedDepJarNames = getDirectlyUsedJars(project, dotFile)
     
     return usedDepJarNames
   }
   
-  protected Set getUsedJars(Project project, File dotFile) {
+  protected Set getDirectlyUsedJars(Project project, File dotFile) {
     Set<String> usedDepJarNames = new HashSet<>()
     
     def lines = dotFile.readLines()
@@ -244,12 +205,20 @@ class UnusedDeps extends DefaultTask {
     return usedDepJarNames
   }
   
-  protected void lookForDep(File dir, Set<String> foundDeps) {
+  protected void lookForDep(File dir, Set<String> depsInDirectUse) {
     dir.eachFile() {
-      Set<String> usedDepJarNames = getUsedJars(project, it)
-      foundDeps.addAll(usedDepJarNames)
-      
+      depsInDirectUse.addAll(getDirectlyUsedJars(project, it))
+    }
+  }
+  
+  protected Project getTopLvlProject(Project proj) {
+    def topLvlProject
+    if (proj.group ==~ /.*?\.lucene(?:\.\w+)?/) {
+      topLvlProject = project.project(":lucene")
+    } else if (proj.group ==~ /.*?\.solr(?:\.\w+)?/) {
+      topLvlProject = project.project(":solr")
     }
+    return topLvlProject
   }
 }
 
diff --git a/lucene/build.gradle b/lucene/build.gradle
index cb8d785..a1d139d 100644
--- a/lucene/build.gradle
+++ b/lucene/build.gradle
@@ -40,6 +40,8 @@ task runJflex() {
 }
 
 subprojects {
-  tasks.create("unusedDeps", org.apache.lucene.gradle.UnusedDeps)
-  unusedDeps.dependsOn packageDist
+  File target = File.createTempDir()
+  tasks.create("jdepsReport", org.apache.lucene.gradle.JdepsReport, target)
+  tasks.create("unusedDeps", org.apache.lucene.gradle.UnusedDeps, target)
+  unusedDeps.dependsOn jdepsReport
 }
diff --git a/lucene/test-framework/build.gradle b/lucene/test-framework/build.gradle
index 8c52c46..e8eb84a 100644
--- a/lucene/test-framework/build.gradle
+++ b/lucene/test-framework/build.gradle
@@ -29,7 +29,6 @@ dependencies {
 	api ("junit:junit")
   api ('org.hamcrest:hamcrest-core')
 	api ("com.carrotsearch.randomizedtesting:randomizedtesting-runner")
-
-  implementation ('commons-codec:commons-codec')
+  api ('commons-codec:commons-codec')
   
 }
diff --git a/solr/build.gradle b/solr/build.gradle
index 9325c49..9ea77bc 100644
--- a/solr/build.gradle
+++ b/solr/build.gradle
@@ -35,5 +35,8 @@ task packageDist(type: org.apache.lucene.gradle.PackageLuceneSolrDist) {
 }
 
 subprojects {
-  tasks.create("unusedDeps", org.apache.lucene.gradle.UnusedDeps)
+  File target = File.createTempDir()
+  tasks.create("jdepsReport", org.apache.lucene.gradle.JdepsReport, target)
+  tasks.create("unusedDeps", org.apache.lucene.gradle.UnusedDeps, target)
+  unusedDeps.dependsOn jdepsReport
 }
diff --git a/solr/contrib/clustering/build.gradle b/solr/contrib/clustering/build.gradle
index 217133f..44e1210 100644
--- a/solr/contrib/clustering/build.gradle
+++ b/solr/contrib/clustering/build.gradle
@@ -24,14 +24,12 @@ dependencies {
   implementation project(':solr:solr-core')
   implementation project(':solr:solr-solrj')
   implementation project(':lucene:lucene-core')
-  implementation project(':lucene:lucene-memory')
   implementation project(':lucene:analysis:lucene-analyzers-common')
   
   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')
diff --git a/solr/contrib/dataimporthandler/build.gradle b/solr/contrib/dataimporthandler/build.gradle
index fe99459..a673846 100644
--- a/solr/contrib/dataimporthandler/build.gradle
+++ b/solr/contrib/dataimporthandler/build.gradle
@@ -41,20 +41,18 @@ dependencies {
   implementation project(':lucene:lucene-join')
   
   implementation ('org.slf4j:slf4j-api') { transitive = false }
-  implementation ('com.google.guava:guava') { transitive = false }
   implementation ('commons-io:commons-io') { transitive = false }
   implementation ('org.apache.httpcomponents:httpclient') { transitive = false }
   implementation ('org.apache.zookeeper:zookeeper') { transitive = false }
   implementation ('io.dropwizard.metrics:metrics-core') { transitive = false }
-  implementation ('commons-codec:commons-codec') { transitive = false }
   
   testImplementation project(':lucene:lucene-test-framework')
   testImplementation project(':solr:solr-test-framework')
   
-  testImplementation 'org.hsqldb:hsqldb'
-  testImplementation 'org.apache.derby:derby'
-  testImplementation 'org.mockito:mockito-core'
-  testImplementation 'net.bytebuddy:byte-buddy'
-  testImplementation 'org.objenesis:objenesis'
+  testImplementation ('org.mockito:mockito-core')
+  testImplementation ('org.hsqldb:hsqldb')
+  testImplementation ('org.apache.derby:derby')
+  testImplementation ('net.bytebuddy:byte-buddy')
+  testImplementation ('org.objenesis:objenesis')
   
 }
\ No newline at end of file
diff --git a/solr/core/build.gradle b/solr/core/build.gradle
index a40a87e..ffd12ca 100644
--- a/solr/core/build.gradle
+++ b/solr/core/build.gradle
@@ -25,10 +25,10 @@ archivesBaseName = 'solr-core'
 
 dependencies {
   
+  runtimeOnly project(':lucene:analysis:lucene-analyzers-phonetic')
   implementation project(':lucene:lucene-core')
   implementation project(':lucene:lucene-codecs')
   implementation project(':lucene:analysis:lucene-analyzers-common')
-  implementation project(':lucene:analysis:lucene-analyzers-phonetic')
   implementation project(':lucene:analysis:lucene-analyzers-kuromoji')
   implementation project(':lucene:analysis:lucene-analyzers-nori')
   implementation project(':lucene:lucene-suggest')