Skip to content

Commit

Permalink
Shade Guava in the sbt build.
Browse files Browse the repository at this point in the history
  • Loading branch information
Marcelo Vanzin committed Aug 6, 2014
1 parent 616998e commit 2fec990
Show file tree
Hide file tree
Showing 3 changed files with 131 additions and 0 deletions.
122 changes: 122 additions & 0 deletions project/Relocator.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,122 @@
/*
* 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 java.io._

import scala.util.matching.Regex

import org.objectweb.asm._
import org.objectweb.asm.commons._
import sbtassembly.Plugin._

class Relocator(prefix: String, shaded: String, includes: Seq[Regex], excludes: Seq[Regex]) {

/**
* Renames a Java class name based on the configured rules for this relocator.
*
* @param name Class name to relocate.
* @return Relocated name (may be same as original).
*/
def rename(name: String): String = {
val javaName = name.replace('/', '.')
if (shouldRename(javaName)) {
val renamed = shaded + javaName.substring(prefix.length())
renamed.replace('.', '/')
} else {
name
}
}

private def shouldRename(name: String) =
name.startsWith(prefix) && isIncluded(name) && !isExcluded(name)

private def isIncluded(name: String) =
includes.isEmpty || !includes.filter { m => m.pattern.matcher(name).matches() }.isEmpty

private def isExcluded(name: String) =
!excludes.isEmpty && !excludes.filter { m => m.pattern.matcher(name).matches() }.isEmpty

}

class RelocatorRemapper(relocators: List[Relocator]) extends Remapper {

override def mapValue(obj: Object) = {
if (obj.isInstanceOf[String]) {
rename(obj.asInstanceOf[String])
} else {
super.mapValue(obj)
}
}

override def map(name: String) = {
rename(name)
}

def rename(name: String): String = {
var result = name
relocators.foreach { r => result = r.rename(result) }
result
}

}

/**
* Tries to emulate part of the class relocation behavior of maven-shade-plugin. Classes that
* should be relocated are moved to a new location, and all classes are passed through the
* remapper so that references to relocated classes are fixed.
*/
class ShadeStrategy(relocators: List[Relocator]) extends MergeStrategy {

private val remapper = new RelocatorRemapper(relocators)

def name = "shade"

override def apply(tempDir: File, path: String, files: Seq[File]) = {
val (file, newPath) =
if (relocators.isEmpty || !files.head.getAbsolutePath().endsWith(".class")) {
(files.head, path)
} else {
val className = path.substring(0, path.length() - ".class".length())
(remap(files.head, tempDir), remapper.rename(className) + ".class")
}
Right(Seq(file -> newPath))
}

private def remap(klass: File, tempDir: File): File = {
var in: Option[FileInputStream] = None
var out: Option[FileOutputStream] = None
try {
in = Some(new FileInputStream(klass))

val writer = new ClassWriter(0)
val visitor= new RemappingClassAdapter(writer, remapper)
val reader = new ClassReader(in.get)
reader.accept(visitor, ClassReader.EXPAND_FRAMES)

val remappedPath = File.createTempFile(klass.getName(), null, tempDir)
out = Some(new FileOutputStream(remappedPath))
out.get.write(writer.toByteArray())
out.get.close()

remappedPath
} finally {
in.foreach { _.close() }
out.foreach { _.close() }
}
}

}
5 changes: 5 additions & 0 deletions project/SparkBuild.scala
Original file line number Diff line number Diff line change
Expand Up @@ -252,12 +252,17 @@ object Assembly {
import sbtassembly.Plugin._
import AssemblyKeys._

private val shade = new ShadeStrategy(List(
new Relocator("com.google", "org.spark-project.guava", Seq("com\\.google\\.common\\..*".r),
Seq("com\\.google\\.common\\.base\\.Optional.*".r))))

lazy val settings = assemblySettings ++ Seq(
test in assembly := {},
jarName in assembly <<= (version, moduleName) map { (v, mName) => mName + "-"+v + "-hadoop" +
Option(System.getProperty("hadoop.version")).getOrElse("1.0.4") + ".jar" },
mergeStrategy in assembly := {
case PathList("org", "datanucleus", xs @ _*) => MergeStrategy.discard
case m if m.endsWith(".class") => shade
case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard
case m if m.toLowerCase.matches("meta-inf.*\\.sf$") => MergeStrategy.discard
case "log4j.properties" => MergeStrategy.discard
Expand Down
4 changes: 4 additions & 0 deletions project/plugins.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -26,3 +26,7 @@ addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.1")
addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.0")

addSbtPlugin("com.cavorite" % "sbt-avro" % "0.3.2")

libraryDependencies += "org.ow2.asm" % "asm" % "5.0.3"

libraryDependencies += "org.ow2.asm" % "asm-commons" % "5.0.3"

0 comments on commit 2fec990

Please sign in to comment.