summaryrefslogtreecommitdiffstats
path: root/standalone-container/src/main/scala
diff options
context:
space:
mode:
Diffstat (limited to 'standalone-container/src/main/scala')
-rw-r--r--standalone-container/src/main/scala/com/yahoo/application/container/impl/ClassLoaderOsgiFramework.scala200
-rw-r--r--standalone-container/src/main/scala/com/yahoo/application/container/impl/StandaloneContainerRunner.scala27
-rw-r--r--standalone-container/src/main/scala/com/yahoo/container/standalone/CloudConfigYinstVariables.scala87
-rw-r--r--standalone-container/src/main/scala/com/yahoo/container/standalone/Converter.scala26
-rw-r--r--standalone-container/src/main/scala/com/yahoo/container/standalone/Environment.scala23
-rw-r--r--standalone-container/src/main/scala/com/yahoo/container/standalone/LocalFileDb.scala71
-rw-r--r--standalone-container/src/main/scala/com/yahoo/container/standalone/StandaloneContainerApplication.scala234
-rw-r--r--standalone-container/src/main/scala/com/yahoo/container/standalone/StandaloneSubscriberFactory.scala78
8 files changed, 746 insertions, 0 deletions
diff --git a/standalone-container/src/main/scala/com/yahoo/application/container/impl/ClassLoaderOsgiFramework.scala b/standalone-container/src/main/scala/com/yahoo/application/container/impl/ClassLoaderOsgiFramework.scala
new file mode 100644
index 00000000000..6d45e6fa8a1
--- /dev/null
+++ b/standalone-container/src/main/scala/com/yahoo/application/container/impl/ClassLoaderOsgiFramework.scala
@@ -0,0 +1,200 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.application.container.impl
+
+import com.yahoo.jdisc.application.{OsgiHeader, OsgiFramework}
+import java.util
+import org.osgi.framework._
+import java.io.InputStream
+import util.concurrent.atomic.AtomicInteger
+import util.jar.JarFile
+import util.{Dictionary, Collections, Hashtable}
+
+import scala.collection.JavaConversions._
+import com.yahoo.container.standalone.StandaloneContainerApplication
+import collection.mutable.ArrayBuffer
+import java.net.{URL, URLClassLoader}
+import org.osgi.framework.wiring._
+import org.osgi.resource.{Wire, Capability, Requirement}
+
+/**
+ * @author tonytv
+ */
+final class ClassLoaderOsgiFramework extends OsgiFramework {
+ private val bundleLocations = new ArrayBuffer[URL]
+ private val bundleList = ArrayBuffer[Bundle](SystemBundleImpl)
+ private var classLoader: ClassLoader = null
+
+ private val nextBundleId = new AtomicInteger(1)
+
+ override def installBundle(bundleLocation: String) = {
+ if (bundleLocation != "") {
+ val url = new URL(bundleLocation)
+ bundleLocations += url
+ bundleList += new JarBundleImpl(url)
+ }
+
+ bundles()
+ }
+
+ def getClassLoader = {
+ if (bundleLocations.isEmpty) {
+ getClass.getClassLoader
+ } else {
+ if(classLoader == null)
+ classLoader = new URLClassLoader(bundleLocations.toArray, getClass.getClassLoader)
+
+ classLoader
+ }
+ }
+
+ override def startBundles(bundles: util.List[Bundle], privileged: Boolean) {}
+
+ override def refreshPackages() {}
+
+ override def bundleContext():BundleContext = BundleContextImpl
+
+ override def bundles() = bundleList
+
+ override def start() {}
+
+ override def stop() {}
+
+ private abstract class BundleImpl extends Bundle {
+ override def getState = Bundle.ACTIVE
+
+ override def start(options: Int) {}
+ override def start() {}
+ override def stop(options: Int) {}
+ override def stop() {}
+ override def update(input: InputStream) {}
+ override def update() {}
+ override def uninstall() {}
+
+ override def getHeaders(locale: String) = getHeaders
+
+ override def getSymbolicName = ClassLoaderOsgiFramework.this.getClass.getName
+ override def getLocation = getSymbolicName
+
+ override def getRegisteredServices = Array[ServiceReference[_]]()
+ override def getServicesInUse = getRegisteredServices
+
+ override def hasPermission(permission: Any) = true
+
+ override def getResource(name: String) = getClassLoader.getResource(name)
+ override def loadClass(name: String) = getClassLoader.loadClass(name)
+ override def getResources(name: String) = getClassLoader.getResources(name)
+
+ override def getEntryPaths(path: String) = throw new UnsupportedOperationException
+ override def getEntry(path: String) = throw new UnsupportedOperationException
+ override def findEntries(path: String, filePattern: String, recurse: Boolean) = throw new UnsupportedOperationException
+
+ override def getLastModified = 1L
+
+ override def getBundleContext = throw new UnsupportedOperationException
+ override def getSignerCertificates(signersType: Int) = Collections.emptyMap()
+
+ override def adapt[A](`type`: Class[A]): A = {
+ if (`type` == classOf[BundleRevision]) BundleRevisionImpl.asInstanceOf[A]
+ else if (`type` == classOf[BundleWiring]) BundleWiringImpl.asInstanceOf[A]
+ else null.asInstanceOf[A]
+ }
+
+ override def getDataFile(filename: String) = null
+ override def compareTo(o: Bundle) = getBundleId compareTo o.getBundleId
+ }
+
+ private object BundleRevisionImpl extends BundleRevision {
+ override def getSymbolicName: String = this.getClass.getName
+ override def getDeclaredRequirements(p1: String): util.List[BundleRequirement] = throw new UnsupportedOperationException
+ override def getVersion: Version = Version.emptyVersion
+ override def getWiring: BundleWiring = BundleWiringImpl
+ override def getDeclaredCapabilities(p1: String): util.List[BundleCapability] = throw new UnsupportedOperationException
+ override def getTypes: Int = 0
+ override def getBundle: Bundle = throw new UnsupportedOperationException
+ override def getCapabilities(p1: String): util.List[Capability] = throw new UnsupportedOperationException
+ override def getRequirements(p1: String): util.List[Requirement] = throw new UnsupportedOperationException
+ }
+
+ private object BundleWiringImpl extends BundleWiring {
+ override def findEntries(p1: String, p2: String, p3: Int): util.List[URL] = ???
+ override def getRequiredResourceWires(p1: String): util.List[Wire] = ???
+ override def getResourceCapabilities(p1: String): util.List[Capability] = ???
+ override def isCurrent: Boolean = ???
+ override def getRequiredWires(p1: String): util.List[BundleWire] = ???
+ override def getCapabilities(p1: String): util.List[BundleCapability] = ???
+ override def getProvidedResourceWires(p1: String): util.List[Wire] = ???
+ override def getProvidedWires(p1: String): util.List[BundleWire] = ???
+ override def getRevision: BundleRevision = ???
+ override def getResourceRequirements(p1: String): util.List[Requirement] = ???
+ override def isInUse: Boolean = ???
+ override def listResources(p1: String, p2: String, p3: Int): util.Collection[String] = ???
+ override def getClassLoader: ClassLoader = ClassLoaderOsgiFramework.this.getClassLoader
+ override def getRequirements(p1: String): util.List[BundleRequirement] = ???
+ override def getResource: BundleRevision = ???
+ override def getBundle: Bundle = ???
+ }
+
+ private object SystemBundleImpl extends BundleImpl {
+ override val getBundleId = 0L
+ override def getVersion = Version.emptyVersion
+ override def getHeaders: Dictionary[String, String] = new Hashtable[String, String](Map(OsgiHeader.APPLICATION -> classOf[StandaloneContainerApplication].getName))
+ }
+
+
+ private class JarBundleImpl(location: URL) extends BundleImpl {
+ override val getBundleId = nextBundleId.getAndIncrement.asInstanceOf[Long]
+
+ private val headers = retrieveHeaders(location)
+
+ override def getHeaders: Dictionary[String, String] = headers
+ override val getSymbolicName = headers.get("Bundle-SymbolicName")
+ override val getVersion = Version.parseVersion(headers.get("Bundle-Version"))
+
+
+ private def retrieveHeaders(location: URL) = {
+ val jarFile = new JarFile(location.getFile)
+ try {
+ val attributes = jarFile.getManifest.getMainAttributes
+ new Hashtable[String, String](attributes.entrySet().map( entry => entry.getKey.toString -> entry.getValue.toString).toMap)
+ } finally {
+ jarFile.close()
+ }
+ }
+ }
+
+ private object BundleContextImpl extends BundleContext {
+ private val bundleImpl = SystemBundleImpl
+
+ override def getProperty(key: String) = null
+ override def getBundle = bundleImpl
+ override def installBundle(location: String, input: InputStream) = throw new UnsupportedOperationException
+ override def installBundle(location: String) = throw new UnsupportedOperationException
+
+ override def getBundle(id: Long) = bundleImpl
+ override def getBundles = Array(bundleImpl)
+ override def getBundle(location: String) = bundleImpl
+
+ override def addServiceListener(listener: ServiceListener, filter: String) {}
+ override def addServiceListener(listener: ServiceListener) {}
+ override def removeServiceListener(listener: ServiceListener) {}
+ override def addBundleListener(listener: BundleListener) {}
+ override def removeBundleListener(listener: BundleListener) {}
+
+ override def addFrameworkListener(listener: FrameworkListener) {}
+ override def removeFrameworkListener(listener: FrameworkListener) {}
+
+ override def registerService(clazzes: Array[String], service: Any, properties: Dictionary[String, _]) = throw new UnsupportedOperationException
+ override def registerService(clazz: String, service: Any, properties: Dictionary[String, _]) = null
+ override def registerService[S](clazz: Class[S], service: S, properties: Dictionary[String, _]) = throw new UnsupportedOperationException
+ override def getServiceReferences(clazz: String, filter: String) = throw new UnsupportedOperationException
+ override def getAllServiceReferences(clazz: String, filter: String) = throw new UnsupportedOperationException
+ override def getServiceReference(clazz: String) = throw new UnsupportedOperationException
+ override def getServiceReference[S](clazz: Class[S]) = throw new UnsupportedOperationException
+ override def getServiceReferences[S](clazz: Class[S], filter: String) = Collections.emptyList()
+ override def getService[S](reference: ServiceReference[S]) = throw new UnsupportedOperationException
+ override def ungetService(reference: ServiceReference[_]) = throw new UnsupportedOperationException
+ override def getDataFile(filename: String) = throw new UnsupportedOperationException
+ override def createFilter(filter: String) = throw new UnsupportedOperationException
+
+ }
+}
diff --git a/standalone-container/src/main/scala/com/yahoo/application/container/impl/StandaloneContainerRunner.scala b/standalone-container/src/main/scala/com/yahoo/application/container/impl/StandaloneContainerRunner.scala
new file mode 100644
index 00000000000..57f60febb26
--- /dev/null
+++ b/standalone-container/src/main/scala/com/yahoo/application/container/impl/StandaloneContainerRunner.scala
@@ -0,0 +1,27 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.application.container.impl
+
+import java.nio.file.Files
+import com.yahoo.text.Utf8
+
+/**
+ * @author tonytv
+ */
+final class StandaloneContainerRunner {
+
+
+
+}
+
+object StandaloneContainerRunner {
+ def createApplicationPackage(servicesXml: String) = {
+ val applicationDir = Files.createTempDirectory("application")
+
+ val servicesXmlFile = applicationDir.resolve("services.xml");
+ var content = servicesXml;
+ if ( ! servicesXml.startsWith("<?xml"))
+ content = """<?xml version="1.0" encoding="utf-8" ?>""" + '\n' + servicesXml
+ Files.write(servicesXmlFile, Utf8.toBytes(content))
+ applicationDir
+ }
+}
diff --git a/standalone-container/src/main/scala/com/yahoo/container/standalone/CloudConfigYinstVariables.scala b/standalone-container/src/main/scala/com/yahoo/container/standalone/CloudConfigYinstVariables.scala
new file mode 100644
index 00000000000..188bedfbc20
--- /dev/null
+++ b/standalone-container/src/main/scala/com/yahoo/container/standalone/CloudConfigYinstVariables.scala
@@ -0,0 +1,87 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.container.standalone
+
+import java.util.Optional
+
+import com.yahoo.vespa.model.container.configserver.option.CloudConfigOptions
+import com.yahoo.vespa.model.container.configserver.option.CloudConfigOptions.ConfigServer
+
+import scala.language.implicitConversions
+import scala.util.Try
+
+/**
+ * @author tonytv
+ */
+class CloudConfigYinstVariables extends CloudConfigOptions {
+ import CloudConfigYinstVariables._
+
+ override val rpcPort = optionalYinstVar[Integer]("port_configserver_rpc", "services")
+ override val allConfigServers = yinstVar("addr_configserver", "services") withDefault Array[ConfigServer]()
+ override val multiTenant = optionalYinstVar[java.lang.Boolean]("multitenant")
+
+ override val zookeeperBarrierTimeout = optionalYinstVar[java.lang.Long]("zookeeper_barrier_timeout")
+ override val sessionLifeTimeSecs = optionalYinstVar[java.lang.Long]("session_lifetime")
+ override val configModelPluginDirs = yinstVar("config_model_plugin_dirs") withDefault Array[String]()
+ override val zookeeperClientPort = optionalYinstVar[Integer]("zookeeper_clientPort")
+ override val zookeeperQuorumPort = optionalYinstVar[Integer]("zookeeper_quoromPort")
+ override val zookeeperElectionPort = optionalYinstVar[Integer]("zookeeper_electionPort")
+ override val payloadCompressionType = optionalYinstVar[java.lang.String]("payload_compression_type")
+ override val environment = optionalYinstVar[java.lang.String]("environment")
+ override val region = optionalYinstVar[java.lang.String]("region")
+ override val defaultFlavor = optionalYinstVar[java.lang.String]("default_flavor")
+ override val defaultAdminFlavor = optionalYinstVar[java.lang.String]("default_admin_flavor")
+ override val defaultContainerFlavor = optionalYinstVar[java.lang.String]("default_container_flavor")
+ override val defaultContentFlavor = optionalYinstVar[java.lang.String]("default_content_flavor")
+ override val useVespaVersionInRequest = optionalYinstVar[java.lang.Boolean]("use_vespa_version_in_request")
+ override val hostedVespa = optionalYinstVar[java.lang.Boolean]("hosted_vespa")
+ override val numParallelTenantLoaders = optionalYinstVar[java.lang.Integer]("num_parallel_tenant_loaders")
+}
+
+object CloudConfigYinstVariables {
+ private class YinstVariable(yinstPkg:String, name: String) {
+ val value = Environment.optionalYinstVariable(yinstPkg + "." + name)
+
+ def withDefault[T](defaultValue: T)(implicit c: Converter[T]) : T = {
+ value map { implicitly[Converter[T]].convert } getOrElse defaultValue
+ }
+ }
+
+ private def yinstVar(setting:String, yinstPkg: String = "cloudconfig_server") = new YinstVariable(yinstPkg, setting)
+
+ private def optionalYinstVar[T](setting:String, yinstPkg: String = "cloudconfig_server")(implicit c: Converter[T]): Optional[T] = {
+ Environment.optionalYinstVariable(yinstPkg + "." + setting) map ( c.convert )
+ }
+
+ implicit val configServerConverter: Converter[Array[ConfigServer]] = new Converter[Array[ConfigServer]] {
+ override def convert(s: String) = {
+ s split "[, ]" filter { !_.isEmpty } map { toConfigServer }
+ }
+ }
+
+ implicit val stringArrayConverter: Converter[Array[String]] = new Converter[Array[String]] {
+ override def convert(s: String) = {
+ s split "[, ]" filter { !_.isEmpty }
+ }
+ }
+
+ private def toConfigServer(hostPort: String): ConfigServer = Try {
+ val (host, portStr) = splitFirst(hostPort, ':')
+ val port = portStr map { _.toInt }
+ new ConfigServer(host, port)
+ }.getOrElse(throw new IllegalArgumentException(s"Invalid config server '$hostPort'"))
+
+ private def splitFirst(string: String, separator: Character): (String, Option[String]) = {
+ val (beginning, endWithSeparator) = string span { _ != separator }
+ (beginning, tailOption(endWithSeparator))
+ }
+
+ def tailOption(s: String) = {
+ if (s.isEmpty) None
+ else Some(s.tail)
+ }
+
+ implicit def toJavaOptional[U <% V, V](option: Option[U]): Optional[V] = option match {
+ case Some(u) => Optional.of(u: V)
+ case None => Optional.empty()
+ }
+}
diff --git a/standalone-container/src/main/scala/com/yahoo/container/standalone/Converter.scala b/standalone-container/src/main/scala/com/yahoo/container/standalone/Converter.scala
new file mode 100644
index 00000000000..443806d014b
--- /dev/null
+++ b/standalone-container/src/main/scala/com/yahoo/container/standalone/Converter.scala
@@ -0,0 +1,26 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.container.standalone
+
+/**
+ * @author tonytv
+ */
+trait Converter[T] {
+ def convert(s: String): T
+}
+
+object Converter {
+ def toConverter[T](f: String => T) = new Converter[T] {
+ override def convert(s: String) = f(s)
+ }
+
+ implicit val intConverter = toConverter(_.toInt)
+ implicit val longConverter = toConverter(_.toLong)
+ implicit val boolConverter = toConverter(_.toBoolean)
+ implicit val stringConverter = toConverter(identity)
+
+ implicit val javaIntegerConverter:Converter[Integer] = toConverter(_.toInt)
+ implicit val javaLongConverter:Converter[java.lang.Long] = toConverter(_.toLong)
+ implicit val javaBooleanConverter:Converter[java.lang.Boolean] = toConverter(_.toBoolean)
+
+
+} \ No newline at end of file
diff --git a/standalone-container/src/main/scala/com/yahoo/container/standalone/Environment.scala b/standalone-container/src/main/scala/com/yahoo/container/standalone/Environment.scala
new file mode 100644
index 00000000000..98671e2addf
--- /dev/null
+++ b/standalone-container/src/main/scala/com/yahoo/container/standalone/Environment.scala
@@ -0,0 +1,23 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.container.standalone
+
+/**
+ * @author tonytv
+ * TODO: copied from standalone-container. Move to separate lib module instead.
+ */
+object Environment {
+ def optionalYinstVariable(name: String) = {
+ env(name.replace(".", "__")).
+ orElse(systemProperty(name)) //for unit testing
+ }
+
+ def yinstVariable(name: String) = {
+ optionalYinstVariable(name).
+ getOrElse {
+ throw new IllegalStateException("Environment variable not set: " + name)
+ }
+ }
+
+ def env(name: String) = Option(System.getenv(name))
+ def systemProperty(name: String) = Option(System.getProperty(name))
+}
diff --git a/standalone-container/src/main/scala/com/yahoo/container/standalone/LocalFileDb.scala b/standalone-container/src/main/scala/com/yahoo/container/standalone/LocalFileDb.scala
new file mode 100644
index 00000000000..7af08a4685d
--- /dev/null
+++ b/standalone-container/src/main/scala/com/yahoo/container/standalone/LocalFileDb.scala
@@ -0,0 +1,71 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.container.standalone
+
+import java.io.File
+import java.lang.reflect.Constructor
+import java.util
+import java.util.concurrent.TimeUnit
+import com.yahoo.config.FileReference
+import com.yahoo.config.application.api.FileRegistry
+import com.yahoo.config.application.api.FileRegistry.Entry
+import com.yahoo.filedistribution.fileacquirer.FileAcquirer
+import com.yahoo.net.HostName
+import scala.collection.JavaConversions._
+
+
+import LocalFileDb._
+import scala.collection.mutable
+import java.nio.file.Path
+
+
+/**
+ * FileAcquirer and FileRegistry working on a local directory.
+ * @author tonytv
+ */
+class LocalFileDb(appPath: Path) extends FileAcquirer with FileRegistry {
+ private val fileReferenceToFile = mutable.Map[FileReference, File]()
+
+ /** *** FileAcquirer overrides *****/
+ def waitFor(reference: FileReference, l: Long, timeUnit: TimeUnit): File = {
+ synchronized {
+ fileReferenceToFile.get(reference).getOrElse {
+ throw new RuntimeException("Invalid file reference " + reference)
+ }
+ }
+ }
+
+ override def shutdown() {}
+
+ /** *** FileRegistry overrides *****/
+ def addFile(relativePath: String): FileReference = {
+ val file = appPath.resolve(relativePath).toFile
+ if (!file.exists) {
+ throw new RuntimeException("The file does not exist: " + file.getPath)
+ }
+
+ val fileReference: FileReference = fileReferenceConstructor.newInstance("LocalFileDb:" + relativePath)
+ fileReferenceToFile.put(fileReference, file)
+ fileReference
+ }
+
+ def fileSourceHost: String =
+ HostName.getLocalhost
+
+ def allRelativePaths: java.util.Set[String] = {
+ new java.util.HashSet(fileReferenceToFile.values.map(_.getPath))
+ }
+
+ override def export(): util.List[Entry] = {
+ new java.util.ArrayList(fileReferenceToFile.keys.map{ (ref: FileReference) => new Entry(fileReferenceToFile.get(ref).get.getPath, ref)})
+ }
+}
+
+object LocalFileDb {
+ private def createFileReferenceConstructor: Constructor[FileReference] = {
+ val method: Constructor[FileReference] = classOf[FileReference].getDeclaredConstructor(classOf[String])
+ method.setAccessible(true)
+ method
+ }
+
+ private val fileReferenceConstructor: Constructor[FileReference] = createFileReferenceConstructor
+}
diff --git a/standalone-container/src/main/scala/com/yahoo/container/standalone/StandaloneContainerApplication.scala b/standalone-container/src/main/scala/com/yahoo/container/standalone/StandaloneContainerApplication.scala
new file mode 100644
index 00000000000..324de2771f4
--- /dev/null
+++ b/standalone-container/src/main/scala/com/yahoo/container/standalone/StandaloneContainerApplication.scala
@@ -0,0 +1,234 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.container.standalone
+
+import com.google.inject.{Key, AbstractModule, Injector, Inject}
+import com.yahoo.config.application.api.{DeployLogger, RuleConfigDeriver, FileRegistry, ApplicationPackage}
+import com.yahoo.config.provision.Zone
+import com.yahoo.jdisc.application.Application
+import com.yahoo.container.jdisc.ConfiguredApplication
+import java.io.{IOException, File}
+import com.yahoo.config.model.test.MockRoot
+import com.yahoo.config.model.application.provider._
+import com.yahoo.vespa.defaults.Defaults
+import com.yahoo.vespa.model.container.xml.{ConfigServerContainerModelBuilder, ManhattanContainerModelBuilder, ContainerModelBuilder}
+import org.w3c.dom.Element
+import com.yahoo.config.model.builder.xml.XmlHelper
+import com.yahoo.vespa.model.container.Container
+import com.yahoo.collections.CollectionUtil.first
+import com.yahoo.vespa.model.builder.xml.dom.VespaDomBuilder
+import com.yahoo.io.IOUtils
+import com.yahoo.container.di.config.SubscriberFactory
+import StandaloneContainerApplication._
+import com.google.inject.name.Names
+import scala.util.Try
+import java.nio.file.{FileSystems, Path, Paths, Files}
+import com.yahoo.config.model.{ConfigModelRepo, ApplicationConfigProducerRoot}
+import scala.collection.JavaConversions._
+import com.yahoo.text.XML
+import com.yahoo.vespa.model.container.xml.ContainerModelBuilder.Networking
+
+import java.lang.{ Boolean => JBoolean }
+import Environment._
+import com.yahoo.config.model.deploy.DeployState
+
+/**
+ * @author tonytv
+ * @author gjoranv
+ */
+class StandaloneContainerApplication @Inject()(injector: Injector) extends Application {
+
+ ConfiguredApplication.ensureVespaLoggingInitialized()
+
+ val applicationPath: Path = injectedApplicationPath.getOrElse(yinstApplicationPath)
+
+ val distributedFiles = new LocalFileDb(applicationPath)
+
+ val configModelRepo = Try { injector.getInstance(Key.get(classOf[ConfigModelRepo], configModelRepoName))}.getOrElse(new ConfigModelRepo)
+
+ val networkingOption = Try {
+ injector.getInstance(Key.get(classOf[JBoolean], Names.named(disableNetworkingAnnotation)))
+ }.map {
+ case JBoolean.TRUE => Networking.disable
+ case JBoolean.FALSE => Networking.enable
+ }.getOrElse(Networking.enable)
+
+ val (modelRoot, container) = withTempDir(
+ preprocessedApplicationDir => createContainerModel(applicationPath, distributedFiles, preprocessedApplicationDir, networkingOption, configModelRepo))
+
+ val configuredApplication = createConfiguredApplication(container)
+
+ def createConfiguredApplication(container: Container): Application = {
+ val augmentedInjector = injector.createChildInjector(new AbstractModule {
+ def configure() {
+ bind(classOf[SubscriberFactory]).toInstance(new StandaloneSubscriberFactory(modelRoot))
+ }
+ })
+
+ System.setProperty("config.id", container.getConfigId) //TODO: DRY
+ augmentedInjector.getInstance(classOf[ConfiguredApplication])
+ }
+
+ def injectedApplicationPath = Try {
+ injector.getInstance(Key.get(classOf[Path], applicationPathName))
+ }.toOption
+
+ def yinstApplicationPath = path(yinstVariable(applicationLocationYinstVariable))
+
+ override def start() {
+ try {
+ com.yahoo.container.Container.get().setCustomFileAcquirer(distributedFiles)
+ configuredApplication.start()
+ }
+ catch {
+ case e: Exception => { com.yahoo.container.Container.resetInstance(); throw e; }
+ }
+ }
+
+ override def stop() {
+ configuredApplication.stop()
+ }
+
+ override def destroy() {
+ com.yahoo.container.Container.resetInstance()
+ configuredApplication.destroy()
+ }
+}
+
+object StandaloneContainerApplication {
+ val packageName = "standalone_jdisc_container"
+ val applicationLocationYinstVariable = s"$packageName.app_location"
+ val deploymentProfileYinstVariable = s"$packageName.deployment_profile"
+ val manhattanHttpPortYinstVariable = s"$packageName.manhattan_http_port"
+
+ val applicationPathName = Names.named(applicationLocationYinstVariable)
+
+ val disableNetworkingAnnotation = "JDisc.disableNetworking"
+ val configModelRepoName = Names.named("ConfigModelRepo")
+ val configDefinitionRepo = new StaticConfigDefinitionRepo()
+
+ val defaultTmpBaseDir = Defaults.getDefaults().underVespaHome("tmp")
+ val tmpDirName = "standalone_container"
+
+ private def withTempDir[T](f: File => T): T = {
+ val tmpDir = createTempDir()
+ try {
+ f(tmpDir)
+ } finally {
+ IOUtils.recursiveDeleteDir(tmpDir)
+ }
+ }
+
+ private def createTempDir(): File = {
+ def getBaseDir: Path = {
+ val tmpBaseDir =
+ if (new File(defaultTmpBaseDir).exists())
+ defaultTmpBaseDir
+ else
+ System.getProperty("java.io.tmpdir")
+
+ Paths.get(tmpBaseDir)
+ }
+
+ val basePath: Path = getBaseDir
+ val tmpDir = Files.createTempDirectory(basePath, tmpDirName)
+ tmpDir.toFile
+ }
+
+ private def validateApplication(applicationPackage: ApplicationPackage, logger: DeployLogger) = {
+ try {
+ applicationPackage.validateXML(logger)
+ } catch {
+ case e: IOException => throw new IllegalArgumentException(e)
+ }
+ }
+
+ def newContainerModelBuilder(networkingOption: Networking): ContainerModelBuilder = {
+ optionalYinstVariable(deploymentProfileYinstVariable) match {
+ case None => new ContainerModelBuilder(true, networkingOption)
+ case Some("manhattan") => new ManhattanContainerModelBuilder(manhattanHttpPort)
+ case Some("configserver") => new ConfigServerContainerModelBuilder(new CloudConfigYinstVariables)
+ case profileName => throw new RuntimeException(s"Invalid deployment profile '$profileName'")
+ }
+ }
+
+ def manhattanHttpPort: Int = {
+ val port = yinstVariable(manhattanHttpPortYinstVariable)
+ Try {
+ Integer.parseInt(port)
+ } filter( _ > 0) getOrElse {
+ throw new RuntimeException(s"$manhattanHttpPortYinstVariable is not a valid port: '$port'")
+ }
+ }
+
+ def createContainerModel(applicationPath: Path,
+ fileRegistry: FileRegistry,
+ preprocessedApplicationDir: File,
+ networkingOption: Networking,
+ configModelRepo: ConfigModelRepo = new ConfigModelRepo): (MockRoot, Container) = {
+ val logger = new BaseDeployLogger
+ val rawApplicationPackage = new FilesApplicationPackage.Builder(applicationPath.toFile).preprocessedDir(preprocessedApplicationDir).build()
+ // TODO: Needed until we get rid of semantic rules
+ val applicationPackage = rawApplicationPackage.preprocess(Zone.defaultZone(), new RuleConfigDeriver {
+ override def derive(ruleBaseDir: String, outputDir: String): Unit = {}
+ }, logger)
+ validateApplication(applicationPackage, logger)
+ val deployState = new DeployState.Builder().
+ applicationPackage(applicationPackage).
+ fileRegistry(fileRegistry).
+ deployLogger(logger).
+ configDefinitionRepo(configDefinitionRepo).
+ build()
+
+ val root = new MockRoot("", deployState)
+ val vespaRoot = new ApplicationConfigProducerRoot(root,
+ "vespa",
+ deployState.getDocumentModel,
+ deployState.getProperties.vespaVersion(),
+ deployState.getProperties.applicationId())
+
+ val spec = containerRootElement(applicationPackage)
+ val containerModel = newContainerModelBuilder(networkingOption).build(deployState, configModelRepo, vespaRoot, spec)
+ containerModel.getCluster().prepare()
+ containerModel.initialize(configModelRepo)
+ val container = first(containerModel.getCluster().getContainers)
+
+ // Always disable rpc server for standalone container. This server will soon be removed anyway.
+ container.setRpcServerEnabled(false)
+ container.setHttpServerEnabled(networkingOption == Networking.enable)
+
+ initializeContainer(container, spec)
+ root.freezeModelTopology()
+ (root, container)
+ }
+
+ def initializeContainer(container: Container, spec: Element) {
+ val host = container.getRoot.getHostSystem.getHost(Container.SINGLENODE_CONTAINER_SERVICESPEC)
+
+ container.setBasePort(VespaDomBuilder.getXmlWantedPort(spec))
+ container.setHostResource(host)
+ container.initService()
+ }
+
+ def getJDiscInServices(element: Element): Element = {
+ def nameAndId(elements: List[Element]): List[String] = {
+ elements map { e => s"${e.getNodeName} id='${e.getAttribute("id")}'" }
+ }
+
+ val jDiscElements = ContainerModelBuilder.configModelIds flatMap { name => XML.getChildren(element, name.getName) }
+ jDiscElements.toList match {
+ case List(e) => e
+ case Nil => throw new RuntimeException("No jdisc element found under services.")
+ case multipleElements: List[Element] => throw new RuntimeException("Found multiple JDisc elements: " + nameAndId(multipleElements).mkString(", "))
+ }
+ }
+
+ def containerRootElement(applicationPackage: ApplicationPackage) : Element = {
+ val element = XmlHelper.getDocument(applicationPackage.getServices).getDocumentElement
+ val nodeName = element.getNodeName
+
+ if (ContainerModelBuilder.configModelIds.map(_.getName).contains(nodeName)) element
+ else getJDiscInServices(element)
+ }
+
+ def path(s: String) = FileSystems.getDefault.getPath(s)
+}
diff --git a/standalone-container/src/main/scala/com/yahoo/container/standalone/StandaloneSubscriberFactory.scala b/standalone-container/src/main/scala/com/yahoo/container/standalone/StandaloneSubscriberFactory.scala
new file mode 100644
index 00000000000..432e5b82946
--- /dev/null
+++ b/standalone-container/src/main/scala/com/yahoo/container/standalone/StandaloneSubscriberFactory.scala
@@ -0,0 +1,78 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.container.standalone
+
+import com.yahoo.config.model.test.MockRoot
+import com.yahoo.config.{ConfigBuilder, ConfigInstance}
+import com.yahoo.container.di.ConfigKeyT
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+import com.yahoo.vespa.config.ConfigKey
+import com.yahoo.container.di.config.{SubscriberFactory, Subscriber}
+import StandaloneSubscriberFactory._
+
+/**
+ * @author tonytv
+ * @author gjoranv
+ */
+class StandaloneSubscriberFactory(root: MockRoot) extends SubscriberFactory {
+ class StandaloneSubscriber(configKeys: Set[ConfigKeyT]) extends Subscriber {
+ override def configChanged =
+ generation == 0
+
+ override def close() {}
+
+ override def config = {
+
+ def getConfig(key: ConfigKeyT) = {
+ val builderWithModelConfig = root.getConfig(newBuilderInstance(key), key.getConfigId)
+
+ require(builderWithModelConfig != null, "Invalid config id " + key.getConfigId )
+ (key.asInstanceOf[ConfigKey[ConfigInstance]], newConfigInstance(builderWithModelConfig))
+ }
+
+ (configKeys map getConfig).toMap.asJava
+ }
+
+ override def waitNextGeneration() = {
+ generation += 1
+
+ if (generation != 0) {
+ while (!Thread.interrupted())
+ Thread.sleep(10000)
+ }
+
+ generation
+ }
+
+ //if waitNextGeneration has not yet been called, -1 should be returned
+ var generation = -1L
+ }
+
+ override def getSubscriber(configKeys: java.util.Set[_ <: ConfigKey[_]]) =
+ new StandaloneSubscriber(configKeys.toSet.asInstanceOf[Set[ConfigKeyT]])
+
+ def reloadActiveSubscribers(generation: Long) {
+ throw new RuntimeException("unsupported")
+ }
+}
+
+object StandaloneSubscriberFactory {
+
+ private def newBuilderInstance(key: ConfigKeyT) =
+ builderClass(key).newInstance()
+
+ private def builderClass(key: ConfigKeyT) = {
+ val nestedClasses = key.getConfigClass.getClasses
+ nestedClasses.
+ filter {_.getName.equals(key.getConfigClass.getName + "$Builder")}.
+ head.
+ asInstanceOf[Class[ConfigInstance.Builder]]
+ }
+
+ private def newConfigInstance(builder: ConfigBuilder) =
+ configClass(builder).getConstructor(builder.getClass).newInstance(builder)
+
+ private def configClass(builder: ConfigBuilder) =
+ builder.getClass.getEnclosingClass.asInstanceOf[Class[ConfigInstance]]
+
+}