diff --git a/build.sbt b/build.sbt index 0ce75e6d..352aec58 100644 --- a/build.sbt +++ b/build.sbt @@ -41,6 +41,7 @@ val userProjects: Seq[ProjectReference] = Seq[ProjectReference]( discoveryAwsApi, discoveryAwsApiAsync, discoveryConsul, + discoveryEureka, discoveryKubernetesApi, discoveryMarathonApi, leaseKubernetes, @@ -105,6 +106,14 @@ lazy val discoveryConsul = pekkoModule("discovery-consul") libraryDependencies := Dependencies.discoveryConsul, mimaPreviousArtifactsSet) +lazy val discoveryEureka = pekkoModule("discovery-eureka") + .enablePlugins(AutomateHeaderPlugin, ReproducibleBuildsPlugin) + .disablePlugins(MimaPlugin) + .settings( + name := "pekko-discovery-eureka", + libraryDependencies := Dependencies.discoveryEureka, + mimaPreviousArtifactsSet) + // gathers all enabled routes and serves them (HTTP or otherwise) lazy val management = pekkoModule("management") .enablePlugins(AutomateHeaderPlugin, ReproducibleBuildsPlugin) diff --git a/discovery-eureka/src/main/resources/reference.conf b/discovery-eureka/src/main/resources/reference.conf new file mode 100644 index 00000000..18aec42d --- /dev/null +++ b/discovery-eureka/src/main/resources/reference.conf @@ -0,0 +1,19 @@ +# SPDX-License-Identifier: Apache-2.0 + +###################################################### +# Apache Pekko Service Discovery Eureka Config # +###################################################### + +pekko.discovery { + eureka { + class = org.apache.pekko.discovery.eureka.EurekaServiceDiscovery + # default eureka scheme + eureka-scheme = "http" + # default eureka host + eureka-host = "127.0.0.1" + # default eureka port + eureka-port = 8761 + # default eureka-path + eureka-path = "eureka" + } +} diff --git a/discovery-eureka/src/main/scala/org/apache/pekko/discovery/eureka/EurekaResponse.scala b/discovery-eureka/src/main/scala/org/apache/pekko/discovery/eureka/EurekaResponse.scala new file mode 100644 index 00000000..a6b947ae --- /dev/null +++ b/discovery-eureka/src/main/scala/org/apache/pekko/discovery/eureka/EurekaResponse.scala @@ -0,0 +1,37 @@ +/* + * 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. + */ + +package org.apache.pekko.discovery.eureka + +import scala.collection.immutable + +object EurekaResponse { + case class Application(name: String, instance: immutable.Seq[Instance]) + case class Instance(hostName: String, app: String, vipAddress: String, ipAddr: Option[String], + status: String, port: PortWrapper, securePort: PortWrapper, dataCenterInfo: Option[DataCenterInfo], + lastDirtyTimestamp: String) + case class Status() + case class PortWrapper(port: Int, enabled: Boolean) + case class DataCenterInfo(name: String = "MyOwn", + clz: String = "com.netflix.appinfo.InstanceInfo$DefaultDataCenterInfo") +} + +import EurekaResponse._ + +case class EurekaResponse(application: Application, errorCode: Option[String]) diff --git a/discovery-eureka/src/main/scala/org/apache/pekko/discovery/eureka/EurekaServiceDiscovery.scala b/discovery-eureka/src/main/scala/org/apache/pekko/discovery/eureka/EurekaServiceDiscovery.scala new file mode 100644 index 00000000..d38b97d5 --- /dev/null +++ b/discovery-eureka/src/main/scala/org/apache/pekko/discovery/eureka/EurekaServiceDiscovery.scala @@ -0,0 +1,97 @@ +/* + * 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. + */ + +package org.apache.pekko.discovery.eureka + +import org.apache.pekko +import pekko.actor.ActorSystem +import pekko.discovery.ServiceDiscovery.{ Resolved, ResolvedTarget } +import pekko.discovery.eureka.EurekaServiceDiscovery.{ pick, targets } +import pekko.discovery.eureka.JsonFormat._ +import pekko.discovery.{ Lookup, ServiceDiscovery } +import pekko.event.{ LogSource, Logging } +import pekko.http.scaladsl.Http +import pekko.http.scaladsl.model.{ HttpRequest, MediaRange, MediaTypes, Uri } +import pekko.http.scaladsl.model.headers._ +import pekko.http.scaladsl.unmarshalling.Unmarshal + +import java.net.InetAddress +import scala.collection.immutable +import scala.concurrent.Future +import scala.concurrent.duration.FiniteDuration +import scala.util.Try + +object EurekaServiceDiscovery { + private[eureka] def pick( + instances: immutable.Seq[EurekaResponse.Instance]): Future[immutable.Seq[EurekaResponse.Instance]] = { + Future.successful(instances.collect { + case instance if instance.status == "UP" => instance + }) + } + + private[eureka] def targets(instances: immutable.Seq[EurekaResponse.Instance]): immutable.Seq[ResolvedTarget] = { + instances.map { instance => + ResolvedTarget( + host = instance.hostName, + port = Some(instance.port.port), + address = instance.ipAddr.flatMap(ip => Try(InetAddress.getByName(ip)).toOption)) + } + } +} + +class EurekaServiceDiscovery(implicit system: ActorSystem) extends ServiceDiscovery { + + import system.dispatcher + + private val log = Logging(system, getClass)(LogSource.fromClass) + private val settings = EurekaSettings(system) + private val (scheme, host, port, path) = + (settings.scheme, settings.host, settings.port, settings.path) + private val http = Http() + + override def lookup(lookup: Lookup, resolveTimeout: FiniteDuration): Future[ServiceDiscovery.Resolved] = { + + val uriPath = Uri.Path.Empty / path / "apps" / lookup.serviceName + val uri = Uri.from(scheme = scheme, host = host, port = port).withPath(uriPath) + val request = HttpRequest(uri = uri, + headers = immutable.Seq(`Accept-Encoding`(HttpEncodings.gzip), Accept(MediaRange(MediaTypes.`application/json`)))) + + log.info("Requesting seed nodes by: {}", request.uri) + + for { + response <- http.singleRequest(request) + entity <- response.entity.toStrict(resolveTimeout) + response <- { + log.debug("Eureka response: [{}]", entity.data.utf8String) + val unmarshalled = Unmarshal(entity).to[EurekaResponse] + unmarshalled.failed.foreach { _ => + log.error( + "Failed to unmarshal Eureka response status [{}], entity: [{}], uri: [{}]", + response.status.value, + entity.data.utf8String, + uri) + } + unmarshalled + } + instances <- pick(response.application.instance) + } yield Resolved(lookup.serviceName, targets(instances)) + + } + +} diff --git a/discovery-eureka/src/main/scala/org/apache/pekko/discovery/eureka/EurekaSettings.scala b/discovery-eureka/src/main/scala/org/apache/pekko/discovery/eureka/EurekaSettings.scala new file mode 100644 index 00000000..340082d9 --- /dev/null +++ b/discovery-eureka/src/main/scala/org/apache/pekko/discovery/eureka/EurekaSettings.scala @@ -0,0 +1,52 @@ +/* + * 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. + */ + +package org.apache.pekko.discovery.eureka + +import org.apache.pekko +import pekko.actor.{ + ActorSystem, + ClassicActorSystemProvider, + ExtendedActorSystem, + Extension, + ExtensionId, + ExtensionIdProvider +} +import pekko.annotation.ApiMayChange + +@ApiMayChange +final class EurekaSettings(system: ExtendedActorSystem) extends Extension { + private val eurekaConfig = system.settings.config.getConfig("pekko.discovery.eureka") + + val scheme: String = eurekaConfig.getString("eureka-scheme") + val host: String = eurekaConfig.getString("eureka-host") + val port: Int = eurekaConfig.getInt("eureka-port") + val path: String = eurekaConfig.getString("eureka-path") +} + +@ApiMayChange +object EurekaSettings extends ExtensionId[EurekaSettings] with ExtensionIdProvider { + override def get(system: ActorSystem): EurekaSettings = super.get(system) + + override def get(system: ClassicActorSystemProvider): EurekaSettings = super.get(system) + + override def lookup: EurekaSettings.type = EurekaSettings + + override def createExtension(system: ExtendedActorSystem): EurekaSettings = new EurekaSettings(system) +} diff --git a/discovery-eureka/src/main/scala/org/apache/pekko/discovery/eureka/JsonFormat.scala b/discovery-eureka/src/main/scala/org/apache/pekko/discovery/eureka/JsonFormat.scala new file mode 100644 index 00000000..51054cbb --- /dev/null +++ b/discovery-eureka/src/main/scala/org/apache/pekko/discovery/eureka/JsonFormat.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * license agreements; and to You under the Apache License, version 2.0: + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * This file is part of the Apache Pekko project, which was derived from Akka. + */ + +/* + * Copyright (C) 2017-2021 Lightbend Inc. + */ + +package org.apache.pekko.discovery.eureka + +import org.apache.pekko.discovery.eureka.EurekaResponse.{ Application, DataCenterInfo, Instance, PortWrapper } +import org.apache.pekko.http.scaladsl.marshallers.sprayjson.SprayJsonSupport +import spray.json._ + +object JsonFormat extends SprayJsonSupport with DefaultJsonProtocol { + implicit val portFormat: JsonFormat[PortWrapper] = new JsonFormat[PortWrapper] { + + override def read(json: JsValue): PortWrapper = { + json.asJsObject.getFields("$", "@enabled") match { + case Seq(JsNumber(port), JsString(enabled)) => PortWrapper(port.toInt, enabled.toBoolean) + case _ => throw DeserializationException("PortWrapper expected") + } + } + + override def write(obj: PortWrapper): JsValue = JsObject( + "$" -> JsNumber(obj.port), + "@enabled" -> JsString(obj.enabled.toString)) + } + implicit val dataCenterInfoFormat: JsonFormat[DataCenterInfo] = new JsonFormat[DataCenterInfo] { + + override def read(json: JsValue): DataCenterInfo = { + json.asJsObject.getFields("name", "@class") match { + case Seq(JsString(name), JsString(clz)) => DataCenterInfo(name, clz) + case _ => throw DeserializationException("DataCenterInfo expected") + } + } + + override def write(obj: DataCenterInfo): JsValue = JsObject( + "name" -> JsString(obj.name), + "@class" -> JsString(obj.clz)) + } + implicit val instanceFormat: JsonFormat[Instance] = jsonFormat9(Instance.apply) + implicit val applicationFormat: JsonFormat[Application] = jsonFormat2(Application.apply) + implicit val rootFormat: RootJsonFormat[EurekaResponse] = jsonFormat2(EurekaResponse.apply) +} diff --git a/discovery-eureka/src/test/resources/logback.xml b/discovery-eureka/src/test/resources/logback.xml new file mode 100644 index 00000000..dfe6fc8f --- /dev/null +++ b/discovery-eureka/src/test/resources/logback.xml @@ -0,0 +1,15 @@ + + + + + System.out + + %date{MM/dd HH:mm:ss} %-5level[%thread] %logger{1} - %m%n%xException + + + + + + + + \ No newline at end of file diff --git a/discovery-eureka/src/test/resources/response.json b/discovery-eureka/src/test/resources/response.json new file mode 100644 index 00000000..aa74b280 --- /dev/null +++ b/discovery-eureka/src/test/resources/response.json @@ -0,0 +1,133 @@ +{ + "application": { + "name": "test", + "instance": [ + { + "instanceId": "test-localhost", + "hostName": "localhost", + "app": "test", + "ipAddr": "127.0.0.1", + "status": "UP", + "overriddenStatus": "UNKNOWN", + "port": { + "$": 8558, + "@enabled": "true" + }, + "securePort": { + "$": 8558, + "@enabled": "true" + }, + "countryId": 1, + "dataCenterInfo": { + "@class": "com.netflix.appinfo.InstanceInfo$DefaultDataCenterInfo", + "name": "MyOwn" + }, + "leaseInfo": { + "renewalIntervalInSecs": 30, + "durationInSecs": 90, + "registrationTimestamp": 1724231438615, + "lastRenewalTimestamp": 1724232638269, + "evictionTimestamp": 0, + "serviceUpTimestamp": 1724231437712 + }, + "metadata": { + "@class": "java.util.Collections$EmptyMap" + }, + "appGroupName": "DEFAULT_GROUP", + "homePageUrl": "http://127.0.0.1:80", + "statusPageUrl": "http://127.0.0.1:8558/health/ready", + "healthCheckUrl": "http://127.0.0.1:8558/health/alive", + "vipAddress": "test", + "secureVipAddress": "test", + "isCoordinatingDiscoveryServer": "false", + "lastUpdatedTimestamp": "1724231438615", + "lastDirtyTimestamp": "1724231437542", + "actionType": "ADDED" + }, + { + "instanceId": "test-not-in-group", + "hostName": "10.0.0.1", + "app": "test", + "ipAddr": "10.0.0.1", + "status": "UP", + "overriddenStatus": "UNKNOWN", + "port": { + "$": 8558, + "@enabled": "true" + }, + "securePort": { + "$": 8558, + "@enabled": "true" + }, + "countryId": 1, + "dataCenterInfo": { + "@class": "com.netflix.appinfo.InstanceInfo$DefaultDataCenterInfo", + "name": "MyOwn" + }, + "leaseInfo": { + "renewalIntervalInSecs": 30, + "durationInSecs": 90, + "registrationTimestamp": 1724231438615, + "lastRenewalTimestamp": 1724232638269, + "evictionTimestamp": 0, + "serviceUpTimestamp": 1724231437712 + }, + "metadata": { + "@class": "java.util.Collections$EmptyMap" + }, + "appGroupName": "other-group", + "homePageUrl": "http://10.0.0.1:80", + "statusPageUrl": "http://10.0.0.1:8558/health/ready", + "healthCheckUrl": "http://10.0.0.1:8558/health/alive", + "vipAddress": "test", + "secureVipAddress": "test", + "isCoordinatingDiscoveryServer": "false", + "lastUpdatedTimestamp": "1724231438615", + "lastDirtyTimestamp": "1724231437542", + "actionType": "ADDED" + }, + { + "instanceId": "test-status-down", + "hostName": "192.168.1.1", + "app": "test", + "ipAddr": "192.168.1.1", + "status": "DOWN", + "overriddenStatus": "UNKNOWN", + "port": { + "$": 8558, + "@enabled": "true" + }, + "securePort": { + "$": 8558, + "@enabled": "true" + }, + "countryId": 1, + "dataCenterInfo": { + "@class": "com.netflix.appinfo.InstanceInfo$DefaultDataCenterInfo", + "name": "MyOwn" + }, + "leaseInfo": { + "renewalIntervalInSecs": 30, + "durationInSecs": 90, + "registrationTimestamp": 1724231438615, + "lastRenewalTimestamp": 1724232638269, + "evictionTimestamp": 0, + "serviceUpTimestamp": 1724231437712 + }, + "metadata": { + "@class": "java.util.Collections$EmptyMap" + }, + "appGroupName": "DEFAULT_GROUP", + "homePageUrl": "http://192.168.1.1:80", + "statusPageUrl": "http://192.168.1.1:8558/health/ready", + "healthCheckUrl": "http://192.168.1.1:8558/health/alive", + "vipAddress": "test", + "secureVipAddress": "test", + "isCoordinatingDiscoveryServer": "false", + "lastUpdatedTimestamp": "1724231438615", + "lastDirtyTimestamp": "1724231437542", + "actionType": "ADDED" + } + ] + } +} diff --git a/discovery-eureka/src/test/scala/org/apache/pekko/discovery/eureka/EurekaServiceDiscoverySpec.scala b/discovery-eureka/src/test/scala/org/apache/pekko/discovery/eureka/EurekaServiceDiscoverySpec.scala new file mode 100644 index 00000000..ddcfd401 --- /dev/null +++ b/discovery-eureka/src/test/scala/org/apache/pekko/discovery/eureka/EurekaServiceDiscoverySpec.scala @@ -0,0 +1,124 @@ +/* + * 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. + */ + +package org.apache.pekko.discovery.eureka + +import org.apache.pekko +import pekko.actor.ActorSystem +import pekko.discovery.ServiceDiscovery.ResolvedTarget +import pekko.discovery.eureka.{ EurekaServiceDiscovery, JsonFormat } +import pekko.testkit.TestKitBase +import com.typesafe.config.ConfigFactory +import org.kiwiproject.eureka.EmbeddedEurekaServer +import org.scalatest.BeforeAndAfterAll +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.matchers.should.Matchers +import org.scalatest.time.{ Millis, Seconds, Span } +import org.scalatest.wordspec.AnyWordSpecLike +import spray.json._ + +import java.net.InetAddress +import scala.concurrent.{ ExecutionContext, Future } +import scala.concurrent.duration.DurationInt +import scala.io.Source +import scala.util.Try + +class EurekaServiceDiscoverySpec + extends AnyWordSpecLike + with Matchers + with BeforeAndAfterAll + with TestKitBase + with ScalaFutures { + + private val embeddedEurekaServer = new EmbeddedEurekaServer() + embeddedEurekaServer.start() + private val testConfig = ConfigFactory.parseString( + s""" + |pekko.discovery.eureka { + | eureka-port = ${embeddedEurekaServer.getEurekaPort()} + |} + |""".stripMargin).withFallback(ConfigFactory.load()) + + override implicit lazy val system: ActorSystem = ActorSystem("test", testConfig) + implicit val ec: ExecutionContext = system.dispatcher + + implicit override val patienceConfig: PatienceConfig = + PatienceConfig(timeout = scaled(Span(30, Seconds)), interval = scaled(Span(50, Millis))) + + override def afterAll(): Unit = { + super.afterAll() + embeddedEurekaServer.stop() + } + + "Eureka Discovery" should { + "work for defaults" in { + val application = "BANK-ACCOUNT" + val host = "acme.com" + embeddedEurekaServer.getRegistry().registerApplication( + application, host, "https://acme-vip.com", "UP") + + val lookupService = new EurekaServiceDiscovery() + val resolved = lookupService.lookup(application, 10.seconds).futureValue + resolved.addresses should contain( + ResolvedTarget( + host = host, + port = Some(7001), + address = None)) + } + + "pick status and group then resolved targets" in { + val data = resourceAsString("response.json") + + val response = JsonFormat.rootFormat.read(data.parseJson) + val instances = response.application.instance + + val resolved = EurekaServiceDiscovery.targets(instances) + + resolved shouldBe List( + ResolvedTarget( + host = "localhost", + port = Some(8558), + address = Try(InetAddress.getByName("127.0.0.1")).toOption), + ResolvedTarget( + host = "10.0.0.1", + port = Some(8558), + address = Try(InetAddress.getByName("10.0.0.1")).toOption), + ResolvedTarget( + host = "192.168.1.1", + port = Some(8558), + address = Try(InetAddress.getByName("192.168.1.1")).toOption) + ) + + val result = for { + picked <- EurekaServiceDiscovery.pick(instances) + resolved <- Future.successful(EurekaServiceDiscovery.targets(picked)) + } yield resolved + + result.futureValue should contain( + ResolvedTarget( + host = "localhost", + port = Some(8558), + address = Try(InetAddress.getByName("127.0.0.1")).toOption)) + } + + } + + private def resourceAsString(name: String): String = + Source.fromInputStream(getClass.getClassLoader.getResourceAsStream(name)).mkString +} diff --git a/management/src/main/scala/org/apache/pekko/management/scaladsl/PekkoManagement.scala b/management/src/main/scala/org/apache/pekko/management/scaladsl/PekkoManagement.scala index a4daa8bc..c1d8b05c 100644 --- a/management/src/main/scala/org/apache/pekko/management/scaladsl/PekkoManagement.scala +++ b/management/src/main/scala/org/apache/pekko/management/scaladsl/PekkoManagement.scala @@ -61,6 +61,7 @@ final class PekkoManagement(implicit private[pekko] val system: ExtendedActorSys productName = "Pekko Management", dependencies = List( "pekko-discovery-consul", + "pekko-discovery-eureka", "pekko-discovery-aws-api", "pekko-discovery-marathon-api", "pekko-discovery-aws-api-async", diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 64f90a48..d5ee17e9 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -75,6 +75,18 @@ object Dependencies { "org.apache.pekko" %% "pekko-slf4j" % pekkoVersion % Test, "ch.qos.logback" % "logback-classic" % logbackVersion % Test) ++ jacksonDatabind ++ jacksonDatatype // consul depends on insecure version of jackson + val discoveryEureka = Seq( + "org.apache.pekko" %% "pekko-actor" % pekkoVersion, + "org.apache.pekko" %% "pekko-discovery" % pekkoVersion, + "org.apache.pekko" %% "pekko-stream" % pekkoVersion, + "org.apache.pekko" %% "pekko-http" % pekkoHttpVersion, + "org.apache.pekko" %% "pekko-http-spray-json" % pekkoHttpVersion, + "org.apache.pekko" %% "pekko-testkit" % pekkoVersion % Test, + "org.apache.pekko" %% "pekko-slf4j" % pekkoVersion % Test, + "org.kiwiproject" % "embedded-eureka" % "1.0.8" % Test, + "org.scalatest" %% "scalatest" % scalaTestVersion % Test, + "ch.qos.logback" % "logback-classic" % logbackVersion % Test) + val discoveryKubernetesApi = Seq( "org.apache.pekko" %% "pekko-actor" % pekkoVersion, "org.apache.pekko" %% "pekko-discovery" % pekkoVersion, diff --git a/scripts/link-validator.conf b/scripts/link-validator.conf index 961422d4..75f1969f 100644 --- a/scripts/link-validator.conf +++ b/scripts/link-validator.conf @@ -42,7 +42,10 @@ site-link-validator { "http://groovy.codehaus.org/" "http://www.mockito.org" "https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html" + "http://glassfish.java.net/public/CDDL+GPL_1_1.html" + "http://jax-rs-spec.java.net" "http://servlet-spec.java.net" + "http://wiki.fasterxml.com/JacksonForCbor" # Occasionally returns a 500 Internal Server Error "http://code.google.com/" ] @@ -82,6 +85,15 @@ site-link-validator { "http://www.slf4j.org" "http://xerces.apache.org/xerces2-j" "http://xml.apache.org/" + "http://antlr.org" + "http://www.antlr.org" + "http://aopalliance.sourceforge.net" + "http://www.eclipse.org/org/documents/edl-v10.php" + "http://www.javassist.org/" + "http://www.mozilla.org/MPL/MPL-1.1.html" + "http://www.stringtemplate.org" + "http://www.xmlpull.org" + "http://x-stream.github.io/" ] }