Add a client library that provides an RPC mechanism for talking to the Corda node.

The RPC mechanism uses message queues and is essentially conventional except for the fact that it supports marshalling observables. An observable encapsulates a stream of ongoing events, and server-side observables sent to the client are automatically bound to message queues and managed by Artemis.
This commit is contained in:
Mike Hearn
2016-08-26 15:31:17 +02:00
parent 25daa7d688
commit 4d83f1489f
21 changed files with 1191 additions and 97 deletions

View File

@ -13,10 +13,8 @@ repositories {
} }
} }
//noinspection GroovyAssignabilityCheck //noinspection GroovyAssignabilityCheck
configurations { configurations {
// we don't want isolated.jar in classPath, since we want to test jar being dynamically loaded as an attachment // we don't want isolated.jar in classPath, since we want to test jar being dynamically loaded as an attachment
runtime.exclude module: 'isolated' runtime.exclude module: 'isolated'
@ -43,6 +41,7 @@ sourceSets {
// build/reports/project/dependencies/index.html for green highlighted parts of the tree. // build/reports/project/dependencies/index.html for green highlighted parts of the tree.
dependencies { dependencies {
compile project(":core")
compile project(':node') compile project(':node')
// Log4J: logging framework (with SLF4J bindings) // Log4J: logging framework (with SLF4J bindings)
@ -55,6 +54,9 @@ dependencies {
compile 'org.reactfx:reactfx:2.0-M5' compile 'org.reactfx:reactfx:2.0-M5'
compile 'org.fxmisc.easybind:easybind:1.0.3' compile 'org.fxmisc.easybind:easybind:1.0.3'
// Artemis Client: ability to connect to an Artemis broker and control it.
compile "org.apache.activemq:artemis-core-client:${artemis_version}"
// Unit testing helpers. // Unit testing helpers.
testCompile 'junit:junit:4.12' testCompile 'junit:junit:4.12'
testCompile "org.assertj:assertj-core:${assertj_version}" testCompile "org.assertj:assertj-core:${assertj_version}"
@ -70,4 +72,4 @@ quasarScan.dependsOn('classes', ':core:classes', ':contracts:classes')
task integrationTest(type: Test) { task integrationTest(type: Test) {
testClassesDir = sourceSets.integrationTest.output.classesDir testClassesDir = sourceSets.integrationTest.output.classesDir
classpath = sourceSets.integrationTest.runtimeClasspath classpath = sourceSets.integrationTest.runtimeClasspath
} }

View File

@ -0,0 +1,139 @@
package com.r3corda.client
import com.google.common.net.HostAndPort
import com.r3corda.client.impl.CordaRPCClientImpl
import com.r3corda.core.ThreadBox
import com.r3corda.node.services.config.NodeSSLConfiguration
import com.r3corda.node.services.messaging.ArtemisMessagingComponent
import com.r3corda.node.services.messaging.CordaRPCOps
import com.r3corda.node.services.messaging.RPCException
import org.apache.activemq.artemis.api.core.ActiveMQNotConnectedException
import org.apache.activemq.artemis.api.core.client.ActiveMQClient
import org.apache.activemq.artemis.api.core.client.ClientSession
import org.apache.activemq.artemis.api.core.client.ClientSessionFactory
import org.slf4j.LoggerFactory
import rx.Observable
import java.io.Closeable
import java.nio.file.Path
import java.time.Duration
import javax.annotation.concurrent.ThreadSafe
import kotlin.concurrent.thread
/**
* An RPC client connects to the specified server and allows you to make calls to the server that perform various
* useful tasks. See the documentation for [proxy] or review the docsite to learn more about how this API works.
*/
@ThreadSafe
class CordaRPCClient(val host: HostAndPort, certificatesPath: Path) : Closeable, ArtemisMessagingComponent(certificatesPath, sslConfig()) {
companion object {
private val rpcLog = LoggerFactory.getLogger("com.r3corda.rpc")
private fun sslConfig(): NodeSSLConfiguration = object : NodeSSLConfiguration {
override val keyStorePassword = "cordacadevpass"
override val trustStorePassword = "trustpass"
}
}
// TODO: Certificate handling for clients needs more work.
private inner class State {
var running = false
lateinit var sessionFactory: ClientSessionFactory
lateinit var session: ClientSession
lateinit var clientImpl: CordaRPCClientImpl
}
private val state = ThreadBox(State())
/**
* An ID that we used to identify this connection on the server side: kind of like a local port number but
* it persists for the lifetime of this process and survives short TCP connection interruptions. Is -1
* until [start] is called.
*/
var myID: Int = -1
private set
private val myAddressPrefix: String get() = "${ArtemisMessagingComponent.CLIENTS_PREFIX}$myID"
/** Opens the connection to the server and registers a JVM shutdown hook to cleanly disconnect. */
@Throws(ActiveMQNotConnectedException::class)
fun start() {
state.locked {
check(!running)
checkStorePasswords() // Check the password.
val serverLocator = ActiveMQClient.createServerLocatorWithoutHA(tcpTransport(ConnectionDirection.OUTBOUND, host.hostText, host.port))
serverLocator.threadPoolMaxSize = 1
sessionFactory = serverLocator.createSessionFactory()
// We use our initial connection ID as the queue namespace.
myID = sessionFactory.connection.id as Int and 0x000000FFFFFF
session = sessionFactory.createSession()
session.start()
clientImpl = CordaRPCClientImpl(session, state.lock, myAddressPrefix)
running = true
// We will use the ID in strings so strip the sign bit.
}
Runtime.getRuntime().addShutdownHook(thread(start = false) {
close()
})
}
/** Shuts down the client and lets the server know it can free the used resources (in a nice way) */
override fun close() {
state.locked {
if (!running) return
session.close()
sessionFactory.close()
running = false
}
}
/**
* Returns a fresh proxy that lets you invoke RPCs on the server. Calls on it block, and if the server throws an
* exception then it will be rethrown on the client. Proxies are thread safe but only one RPC can be in flight at
* once. If you'd like to perform multiple RPCs in parallel, use this function multiple times to get multiple
* proxies.
*
* Creation of a proxy is a somewhat expensive operation that involves calls to the server, so if you want to do
* calls from many threads at once you should cache one proxy per thread and reuse them. This function itself is
* thread safe though so requires no extra synchronisation.
*
* RPC sends and receives are logged on the com.r3corda.rpc logger.
*
* By default there are no timeouts on calls. RPCs can survive temporary losses or changes in connectivity,
* like switching between wifi networks. You can specify a timeout on the level of a proxy. If a call times
* out it will throw [RPCException.Deadline].
*
* The [CordaRPCOps] defines what client RPCs are available. If an RPC returns an [Observable] anywhere in the
* object graph returned then the server-side observable is transparently linked to a messaging queue, and that
* queue linked to another observable on the client side here. *You are expected to use it*. The server will begin
* buffering messages immediately that it will expect you to drain by subscribing to the returned observer. You can
* opt-out of this by simply casting the [Observable] to [Closeable] or [AutoCloseable] and then calling the close
* method on it. You don't have to explicitly close the observable if you actually subscribe to it: it will close
* itself and free up the server-side resources either when the client or JVM itself is shutdown, or when there are
* no more subscribers to it. Once all the subscribers to a returned observable are unsubscribed, the observable is
* closed and you can't then re-subscribe again: you'll have to re-request a fresh observable with another RPC.
*
* The proxy and linked observables consume some small amount of resources on the server. It's OK to just exit your
* process and let the server clean up, but in a long running process where you only need something for a short
* amount of time it is polite to cast the objects to [Closeable] or [AutoCloseable] and close it when you are done.
* Finalizers are in place to warn you if you lose a reference to an unclosed proxy or observable.
*
* @throws RPCException if the server version is too low or if the server isn't reachable within the given time.
*/
@Throws(RPCException::class)
fun proxy(timeout: Duration? = null, minVersion: Int = 0): CordaRPCOps {
return state.locked {
check(running) { "Client must have been started first" }
clientImpl.proxyFor(CordaRPCOps::class.java, timeout, minVersion)
}
}
private fun finalize() {
state.locked {
if (running) {
rpcLog.warn("A CordaMQClient is being finalised whilst still running, did you forget to call close?")
close()
}
}
}
}

View File

@ -0,0 +1,291 @@
package com.r3corda.client.impl
import com.esotericsoftware.kryo.Kryo
import com.esotericsoftware.kryo.KryoException
import com.esotericsoftware.kryo.Serializer
import com.esotericsoftware.kryo.io.Input
import com.esotericsoftware.kryo.io.Output
import com.google.common.cache.CacheBuilder
import com.r3corda.client.CordaRPCClient
import com.r3corda.core.ErrorOr
import com.r3corda.core.bufferUntilSubscribed
import com.r3corda.core.random63BitValue
import com.r3corda.core.serialization.deserialize
import com.r3corda.core.serialization.serialize
import com.r3corda.core.utilities.debug
import com.r3corda.core.utilities.trace
import com.r3corda.node.services.messaging.*
import org.apache.activemq.artemis.api.core.ActiveMQObjectClosedException
import org.apache.activemq.artemis.api.core.client.ClientConsumer
import org.apache.activemq.artemis.api.core.client.ClientMessage
import org.apache.activemq.artemis.api.core.client.ClientProducer
import org.apache.activemq.artemis.api.core.client.ClientSession
import rx.Observable
import rx.subjects.PublishSubject
import java.io.Closeable
import java.lang.reflect.InvocationHandler
import java.lang.reflect.Method
import java.lang.reflect.Proxy
import java.time.Duration
import java.util.*
import java.util.concurrent.locks.ReentrantLock
import javax.annotation.concurrent.GuardedBy
import javax.annotation.concurrent.ThreadSafe
import kotlin.concurrent.withLock
import kotlin.reflect.jvm.javaMethod
/**
* Core RPC engine implementation, you should be looking at [CordaRPCClient].
*
* @suppress
*/
class CordaRPCClientImpl(private val session: ClientSession,
private val sessionLock: ReentrantLock,
private val myAddressPrefix: String) {
companion object {
private val closeableCloseMethod = Closeable::close.javaMethod
private val autocloseableCloseMethod = AutoCloseable::close.javaMethod
}
/**
* Builds a proxy for the given type, which must descend from [RPCOps].
*
* @see CordaRPCClient.proxy for more information about how to use the proxies.
*/
fun <T : RPCOps> proxyFor(rpcInterface: Class<T>, timeout: Duration? = null, minVersion: Int = 0): T {
sessionLock.withLock {
if (producer == null)
producer = session.createProducer()
}
val proxyImpl = RPCProxyHandler(timeout)
@Suppress("UNCHECKED_CAST")
val proxy = Proxy.newProxyInstance(rpcInterface.classLoader, arrayOf(rpcInterface, Closeable::class.java), proxyImpl) as T
proxyImpl.serverProtocolVersion = proxy.protocolVersion
if (minVersion > proxyImpl.serverProtocolVersion)
throw RPCException("Requested minimum protocol version $minVersion is higher than the server's supported protocol version (${proxyImpl.serverProtocolVersion})")
return proxy
}
//////////////////////////////////////////////////////////////////////////////////////////////////////////////////
//
//region RPC engine
//
// You can find docs on all this in the api doc for the proxyFor method, and in the docsite.
// Utility to quickly suck out the contents of an Artemis message. There's probably a more efficient way to
// do this.
private fun <T : Any> ClientMessage.deserialize(kryo: Kryo): T = ByteArray(bodySize).apply { bodyBuffer.readBytes(this) }.deserialize(kryo)
@GuardedBy("sessionLock")
private val addressToQueueObservables = CacheBuilder.newBuilder().build<String, QueuedObservable>()
private var producer: ClientProducer? = null
private inner class ObservableDeserializer(private val qName: String,
private val rpcName: String,
private val rpcLocation: Throwable) : Serializer<Observable<Any>>() {
override fun read(kryo: Kryo, input: Input, type: Class<Observable<Any>>): Observable<Any> {
val handle = input.readInt(true)
return sessionLock.withLock {
var ob = addressToQueueObservables.getIfPresent(qName)
if (ob == null) {
ob = QueuedObservable(qName, rpcName, rpcLocation, this)
addressToQueueObservables.put(qName, ob)
}
val result = ob.getForHandle(handle)
rpcLog.trace { "Deserializing and connecting a new observable for $rpcName on $qName: $result" }
result
}
}
override fun write(kryo: Kryo, output: Output, `object`: Observable<Any>) {
throw UnsupportedOperationException("not implemented")
}
}
/**
* The proxy class returned to the client is auto-generated on the fly by the java.lang.reflect Proxy
* infrastructure. The JDK Proxy class writes bytecode into memory for a class that implements the requested
* interfaces and then routes all method calls to the invoke method below in a conveniently reified form.
* We can then easily take the data about the method call and turn it into an RPC. This avoids the need
* for the compile-time code generation which is so common in RPC systems.
*/
@ThreadSafe
private inner class RPCProxyHandler(private val timeout: Duration?) : InvocationHandler, Closeable {
private val proxyAddress = "$myAddressPrefix.rpc.responses.${random63BitValue()}"
private val consumer: ClientConsumer
var serverProtocolVersion = 0
init {
consumer = sessionLock.withLock{
session.createTemporaryQueue(proxyAddress, proxyAddress)
session.createConsumer(proxyAddress)
}
}
@Synchronized
override fun invoke(proxy: Any, method: Method, args: Array<out Any>?): Any? {
if (isCloseInvocation(method)) {
close()
return null
}
if (method.name == "toString" && args == null)
return "Client RPC proxy"
if (consumer.isClosed)
throw RPCException("RPC Proxy is closed")
// All invoked methods on the proxy end up here.
val location = Throwable()
rpcLog.trace {
val argStr = args?.joinToString() ?: ""
"-> RPC -> ${method.name}($argStr): ${method.returnType}"
}
checkMethodVersion(method)
// sendRequest may return a reconfigured Kryo if the method returns observables.
val kryo: Kryo = sendRequest(args, location, method) ?: createRPCKryo()
val next = receiveResponse(kryo, method, timeout)
rpcLog.trace { "<- RPC <- ${method.name} = $next" }
return unwrapOrThrow(next)
}
@Suppress("PLATFORM_CLASS_MAPPED_TO_KOTLIN")
private fun unwrapOrThrow(next: ErrorOr<*>): Any? {
val ex = next.error
if (ex != null) {
// Replace the stack trace because that's an implementation detail of the server that isn't so
// helpful to the user who wants to see where the error was on their side, and serialising stack
// frame objects is a bit annoying. We slice it here to avoid the invoke() machinery being exposed.
// The resulting exception looks like it was thrown from inside the called method.
(ex as java.lang.Throwable).stackTrace = java.lang.Throwable().stackTrace.let { it.sliceArray(1..it.size - 1) }
throw ex
} else {
return next.value
}
}
private fun receiveResponse(kryo: Kryo, method: Method, timeout: Duration?): ErrorOr<*> {
val artemisMessage: ClientMessage =
if (timeout == null)
consumer.receive() ?: throw ActiveMQObjectClosedException()
else
consumer.receive(timeout.toMillis()) ?: throw RPCException.DeadlineExceeded(method.name)
artemisMessage.acknowledge()
val next = artemisMessage.deserialize<ErrorOr<*>>(kryo)
return next
}
private fun sendRequest(args: Array<out Any>?, location: Throwable, method: Method): Kryo? {
// We could of course also check the return type of the method to see if it's Observable, but I'd
// rather haved the annotation be used consistently.
val returnsObservables = method.isAnnotationPresent(RPCReturnsObservables::class.java)
sessionLock.withLock {
val msg = createMessage(method)
val kryo = if (returnsObservables) maybePrepareForObservables(location, method, msg) else null
val argsArray = args ?: Array<Any?>(0) { null }
val serializedBytes = try {
argsArray.serialize()
} catch (e: KryoException) {
throw RPCException("Could not serialize RPC arguments", e)
}
msg.writeBodyBufferBytes(serializedBytes.bits)
producer!!.send(ArtemisMessagingComponent.RPC_REQUESTS_QUEUE, msg)
return kryo
}
}
private fun maybePrepareForObservables(location: Throwable, method: Method, msg: ClientMessage): Kryo {
// Create a temporary queue just for the emissions on any observables that are returned.
val qName = "$myAddressPrefix.rpc.observations.${random63BitValue()}"
session.createTemporaryQueue(qName, qName)
msg.putStringProperty(ClientRPCRequestMessage.OBSERVATIONS_TO, qName)
// And make sure that we deserialise observable handles so that they're linked to the right
// queue. Also record a bit of metadata for debugging purposes.
return createRPCKryo(observableSerializer = ObservableDeserializer(qName, method.name, location))
}
private fun createMessage(method: Method): ClientMessage {
return session.createMessage(false).apply {
putStringProperty(ClientRPCRequestMessage.METHOD_NAME, method.name)
putStringProperty(ClientRPCRequestMessage.REPLY_TO, proxyAddress)
}
}
private fun checkMethodVersion(method: Method) {
val methodVersion = method.getAnnotation(RPCSinceVersion::class.java)?.version ?: 0
if (methodVersion > serverProtocolVersion)
throw UnsupportedOperationException("Method ${method.name} was added in RPC protocol version $methodVersion but the server is running $serverProtocolVersion")
}
private fun isCloseInvocation(method: Method) = method == closeableCloseMethod || method == autocloseableCloseMethod
override fun close() {
consumer.close()
sessionLock.withLock { session.deleteQueue(proxyAddress) }
}
override fun toString() = "Corda RPC Proxy listening on queue $proxyAddress"
}
/**
* When subscribed to, starts consuming from the given queue name and demultiplexing the observables being
* sent to it. The server queue is moved into in-memory buffers (one per attached server-side observable)
* until drained through a subscription. When the subscriptions are all gone, the server-side queue is deleted.
*/
@ThreadSafe
private inner class QueuedObservable(private val qName: String,
private val rpcName: String,
private val rpcLocation: Throwable,
private val observableDeserializer: ObservableDeserializer) {
private val root = PublishSubject.create<MarshalledObservation>()
private val rootShared = root.doOnUnsubscribe { close() }.share()
// This could be made more efficient by using a specialised IntMap
private val observables = HashMap<Int, Observable<Any>>()
private var consumer: ClientConsumer? = sessionLock.withLock { session.createConsumer(qName) }.setMessageHandler { deliver(it) }
@Synchronized
fun getForHandle(handle: Int): Observable<Any> {
return observables.getOrPut(handle) {
rootShared.filter { it.forHandle == handle }.map { it.what }.dematerialize<Any>().bufferUntilSubscribed().share()
}
}
private fun deliver(msg: ClientMessage) {
msg.acknowledge()
val kryo = createRPCKryo(observableSerializer = observableDeserializer)
val received: MarshalledObservation = msg.deserialize(kryo)
rpcLog.debug { "<- Observable [$rpcName] <- Received $received" }
synchronized(this) {
// Force creation of the buffer if it doesn't already exist.
getForHandle(received.forHandle)
root.onNext(received)
}
}
@Synchronized
fun close() {
rpcLog.debug("Closing queue observable for call to $rpcName : $qName")
consumer?.close()
consumer = null
sessionLock.withLock { session.deleteQueue(qName) }
}
@Suppress("UNUSED")
fun finalize() {
val c = synchronized(this) { consumer }
if (c != null) {
rpcLog.warn("A hot observable returned from an RPC ($rpcName) was never subscribed to or explicitly closed. " +
"This wastes server-side resources because it was queueing observations for retrieval. " +
"It is being closed now, but please adjust your code to cast the observable to AutoCloseable and then close it explicitly.", rpcLocation)
c.close()
}
}
}
//endregion
}

View File

@ -0,0 +1,198 @@
package com.r3corda.client
import com.r3corda.client.impl.CordaRPCClientImpl
import com.r3corda.core.serialization.SerializedBytes
import com.r3corda.core.utilities.LogHelper
import com.r3corda.node.services.messaging.*
import com.r3corda.node.utilities.AffinityExecutor
import org.apache.activemq.artemis.api.core.SimpleString
import org.apache.activemq.artemis.api.core.TransportConfiguration
import org.apache.activemq.artemis.api.core.client.ActiveMQClient
import org.apache.activemq.artemis.api.core.client.ClientProducer
import org.apache.activemq.artemis.api.core.client.ClientSession
import org.apache.activemq.artemis.core.config.impl.ConfigurationImpl
import org.apache.activemq.artemis.core.remoting.impl.invm.InVMAcceptorFactory
import org.apache.activemq.artemis.core.remoting.impl.invm.InVMConnectorFactory
import org.apache.activemq.artemis.core.server.embedded.EmbeddedActiveMQ
import org.assertj.core.api.Assertions.assertThat
import org.junit.After
import org.junit.Before
import org.junit.Test
import rx.Observable
import rx.subjects.PublishSubject
import java.io.Closeable
import java.util.concurrent.CountDownLatch
import java.util.concurrent.LinkedBlockingQueue
import java.util.concurrent.locks.ReentrantLock
import kotlin.test.assertEquals
import kotlin.test.assertFailsWith
import kotlin.test.assertTrue
class ClientRPCInfrastructureTests {
// TODO: Test that timeouts work
lateinit var artemis: EmbeddedActiveMQ
lateinit var serverSession: ClientSession
lateinit var clientSession: ClientSession
lateinit var producer: ClientProducer
lateinit var serverThread: AffinityExecutor.ServiceAffinityExecutor
lateinit var proxy: ITestOps
@Before
fun setup() {
// Set up an in-memory Artemis with an RPC requests queue.
artemis = EmbeddedActiveMQ()
artemis.setConfiguration(ConfigurationImpl().apply {
acceptorConfigurations = setOf(TransportConfiguration(InVMAcceptorFactory::class.java.name))
isSecurityEnabled = false
isPersistenceEnabled = false
})
artemis.start()
val serverLocator = ActiveMQClient.createServerLocatorWithoutHA(TransportConfiguration(InVMConnectorFactory::class.java.name))
val sessionFactory = serverLocator.createSessionFactory()
serverSession = sessionFactory.createSession()
serverSession.start()
serverSession.createTemporaryQueue(ArtemisMessagingComponent.RPC_REQUESTS_QUEUE, ArtemisMessagingComponent.RPC_REQUESTS_QUEUE)
producer = serverSession.createProducer()
val dispatcher = object : RPCDispatcher(TestOps()) {
override fun send(bits: SerializedBytes<*>, toAddress: String) {
val msg = serverSession.createMessage(false)
msg.writeBodyBufferBytes(bits.bits)
producer.send(toAddress, msg)
}
}
serverThread = AffinityExecutor.ServiceAffinityExecutor("unit-tests-rpc-dispatch-thread", 1)
val serverConsumer = serverSession.createConsumer(ArtemisMessagingComponent.RPC_REQUESTS_QUEUE)
serverSession.createTemporaryQueue("activemq.notifications", "rpc.qremovals", "_AMQ_NotifType = 'BINDING_REMOVED'")
val serverNotifConsumer = serverSession.createConsumer("rpc.qremovals")
dispatcher.start(serverConsumer, serverNotifConsumer, serverThread)
clientSession = sessionFactory.createSession()
clientSession.start()
LogHelper.setLevel("+com.r3corda.rpc"/*, "+org.apache.activemq"*/)
proxy = CordaRPCClientImpl(clientSession, ReentrantLock(), "tests").proxyFor(ITestOps::class.java)
}
@After
fun shutdown() {
(proxy as Closeable).close()
clientSession.stop()
serverSession.stop()
artemis.stop()
serverThread.shutdownNow()
}
interface ITestOps : RPCOps {
@Throws(IllegalArgumentException::class)
fun barf()
fun void()
fun someCalculation(str: String, num: Int): String
@RPCReturnsObservables
fun makeObservable(): Observable<Int>
@RPCReturnsObservables
fun makeComplicatedObservable(): Observable<Pair<String, Observable<String>>>
@RPCSinceVersion(2)
fun addedLater()
}
lateinit var complicatedObservable: Observable<Pair<String, Observable<String>>>
inner class TestOps : ITestOps {
override val protocolVersion = 1
override fun barf() {
throw IllegalArgumentException("Barf!")
}
override fun void() { }
override fun someCalculation(str: String, num: Int) = "$str $num"
override fun makeObservable(): Observable<Int> {
return Observable.just(1, 2, 3, 4)
}
override fun makeComplicatedObservable() = complicatedObservable
override fun addedLater() {
throw UnsupportedOperationException("not implemented")
}
}
@Test
fun simpleRPCs() {
// Does nothing, doesn't throw.
proxy.void()
assertEquals("Barf!", assertFailsWith<IllegalArgumentException> {
proxy.barf()
}.message)
assertEquals("hi 5", proxy.someCalculation("hi", 5))
}
@Test
fun simpleObservable() {
// This tests that the observations are transmitted correctly, also completion is transmitted.
val observations = proxy.makeObservable().toBlocking().toIterable().toList()
assertEquals(listOf(1, 2, 3, 4), observations)
}
@Test
fun complexObservables() {
// This checks that we can return an object graph with complex usage of observables, like an observable
// that emits objects that contain more observables.
val serverQuotes = PublishSubject.create<Pair<String, Observable<String>>>()
val unsubscribeLatch = CountDownLatch(1)
complicatedObservable = serverQuotes.asObservable().doOnUnsubscribe { unsubscribeLatch.countDown() }
val twainQuotes = "Mark Twain" to Observable.just(
"I have never let my schooling interfere with my education.",
"Clothes make the man. Naked people have little or no influence on society."
)
val wildeQuotes = "Oscar Wilde" to Observable.just(
"I can resist everything except temptation.",
"Always forgive your enemies - nothing annoys them so much."
)
val clientQuotes = LinkedBlockingQueue<String>()
val clientObs = proxy.makeComplicatedObservable()
val subscription = clientObs.subscribe {
val name = it.first
it.second.subscribe {
clientQuotes += "Quote by $name: $it"
}
}
assertEquals(1, clientSession.addressQuery(SimpleString("tests.rpc.observations.#")).queueNames.size)
assertThat(clientQuotes).isEmpty()
serverQuotes.onNext(twainQuotes)
assertEquals("Quote by Mark Twain: I have never let my schooling interfere with my education.", clientQuotes.take())
assertEquals("Quote by Mark Twain: Clothes make the man. Naked people have little or no influence on society.", clientQuotes.take())
serverQuotes.onNext(wildeQuotes)
assertEquals("Quote by Oscar Wilde: I can resist everything except temptation.", clientQuotes.take())
assertEquals("Quote by Oscar Wilde: Always forgive your enemies - nothing annoys them so much.", clientQuotes.take())
assertTrue(serverQuotes.hasObservers())
subscription.unsubscribe()
unsubscribeLatch.await()
assertEquals(0, clientSession.addressQuery(SimpleString("tests.rpc.observations.#")).queueNames.size)
}
@Test
fun versioning() {
assertFailsWith<UnsupportedOperationException> { proxy.addedLater() }
}
}

View File

@ -8,6 +8,8 @@ import com.google.common.util.concurrent.MoreExecutors
import com.google.common.util.concurrent.SettableFuture import com.google.common.util.concurrent.SettableFuture
import com.r3corda.core.crypto.newSecureRandom import com.r3corda.core.crypto.newSecureRandom
import org.slf4j.Logger import org.slf4j.Logger
import rx.Observable
import rx.subjects.UnicastSubject
import java.io.BufferedInputStream import java.io.BufferedInputStream
import java.io.InputStream import java.io.InputStream
import java.math.BigDecimal import java.math.BigDecimal
@ -262,4 +264,15 @@ data class ErrorOr<out A> private constructor(val value: A?, val error: Throwabl
// Monad // Monad
fun <B> bind(function: (A) -> ErrorOr<B>) = value?.let(function) ?: ErrorOr.of(error!!) fun <B> bind(function: (A) -> ErrorOr<B>) = value?.let(function) ?: ErrorOr.of(error!!)
}
/**
* Returns an observable that buffers events until subscribed.
*
* @see UnicastSubject
*/
fun <T> Observable<T>.bufferUntilSubscribed(): Observable<T> {
val subject = UnicastSubject.create<T>()
val subscription = subscribe(subject)
return subject.doOnUnsubscribe { subscription.unsubscribe() }
} }

View File

@ -46,7 +46,6 @@ class Wallet(val states: Iterable<StateAndRef<ContractState>>) {
* other transactions observed, then the changes are observed "net" of those. * other transactions observed, then the changes are observed "net" of those.
*/ */
data class Update(val consumed: Set<StateRef>, val produced: Set<StateAndRef<ContractState>>) { data class Update(val consumed: Set<StateRef>, val produced: Set<StateAndRef<ContractState>>) {
/** /**
* Combine two updates into a single update with the combined inputs and outputs of the two updates but net * Combine two updates into a single update with the combined inputs and outputs of the two updates but net
* any outputs of the left-hand-side (this) that are consumed by the inputs of the right-hand-side (rhs). * any outputs of the left-hand-side (this) that are consumed by the inputs of the right-hand-side (rhs).
@ -61,6 +60,17 @@ class Wallet(val states: Iterable<StateAndRef<ContractState>>) {
rhs.produced + produced.filter { it.ref !in rhs.consumed }) rhs.produced + produced.filter { it.ref !in rhs.consumed })
return combined return combined
} }
override fun toString(): String {
val sb = StringBuilder()
sb.appendln("${consumed.size} consumed, ${produced.size} produced")
sb.appendln("")
sb.appendln("Produced:")
produced.forEach {
sb.appendln("${it.ref}: ${it.state}")
}
return sb.toString()
}
} }
companion object { companion object {

View File

@ -409,7 +409,6 @@ class NoReferencesSerializer<T>(val baseSerializer: Serializer<T>) : Serializer<
* Improvement to the builtin JavaSerializer by honouring the [Kryo.getReferences] setting. * Improvement to the builtin JavaSerializer by honouring the [Kryo.getReferences] setting.
*/ */
object ReferencesAwareJavaSerializer : JavaSerializer() { object ReferencesAwareJavaSerializer : JavaSerializer() {
override fun write(kryo: Kryo, output: Output, obj: Any) { override fun write(kryo: Kryo, output: Output, obj: Any) {
if (kryo.references) { if (kryo.references) {
super.write(kryo, output, obj) super.write(kryo, output, obj)

75
docs/source/clientrpc.rst Normal file
View File

@ -0,0 +1,75 @@
Client RPC
==========
There are multiple ways to interact with a node from a *client program*, but if your client is written in a JVM
compatible language the easiest way to do so is using the client library. The library connects to your running
node using a message queue protocol and then provides a simple RPC interface to interact with it. You make calls
on a Java object as normal, and the marshalling back and forth is handled for you.
The starting point for the client library is the `CordaRPCClient`_ class. This provides a ``proxy`` method that
returns an implementation of the `CordaRPCOps`_ interface. A timeout parameter can be specified, and observables that
are returned by RPCs can be subscribed to in order to receive an ongoing stream of updates from the node. More
detail on how to use this is provided in the docs for the proxy method.
.. warning:: The returned object is somewhat expensive to create and consumes a small amount of server side
resources. When you're done with it, cast it to ``Closeable`` or ``AutoCloseable`` and close it. Don't create
one for every call you make - create a proxy and reuse it.
Observables
-----------
The RPC system handles observables in a special way. When a method returns an observable, whether directly or
as a sub-object of the response object graph, an observable is created on the client to match the one on the
server. Objects emitted by the server-side observable are pushed onto a queue which is then drained by the client.
The returned observable may even emit object graphs with even more observables in them, and it all works as you
would expect.
This feature comes with a cost: the server must queue up objects emitted by the server-side observable until you
download them. Therefore RPCs that use this feature are marked with the ``@RPCReturnsObservables`` annotation, and
you are expected to subscribe to all the observables returned. If you don't want an observable then subscribe
then unsubscribe immediately to clear the buffers and indicate that you aren't interested. If your app quits then
server side resources will be freed automatically.
When all the observables returned by an RPC are unsubscribed on the client side, that unsubscription propagates
through to the server where the corresponding server-side observables are also unsubscribed.
.. warning:: If you leak an observable or proxy on the client side and it gets garbage collected, you will get
a warning printed to the logs and the proxy will be closed for you. But don't rely on this, as garbage
collection is non-deterministic.
Versioning
----------
The client RPC protocol is versioned with a simple incrementing integer. When a proxy is created the server is
queried for its protocol version, and you can specify your minimum requirement. Methods added in later versions
are tagged with the ``@RPCSinceVersion`` annotation. If you try to use a method that the server isn't advertising
support of, an ``UnsupportedOperationException`` is thrown. If you want to know the version of the server, just
use the ``protocolVersion`` property (i.e. ``getProtocolVersion`` in Java).
Thread safety
-------------
A proxy is thread safe, blocking, and will only allow a single RPC to be in flight at once. Any observables that
are returned and you subscribe to will have objects emitted on a background thread. Observables returned as part
of one RPC and observables returned from another may have their callbacks invoked in parallel, but observables
returned as part of the same specific RPC invocation are processed serially and will not be invoked in parallel.
If you want to make multiple calls to the server in parallel you can do that by creating multiple proxies, but
be aware that the server itself may *not* process your work in parallel even if you make your requests that way.
Error handling
--------------
If something goes wrong with the RPC infrastructure itself, an ``RPCException`` is thrown. If you call a method that
requires a higher version of the protocol than the server supports, ``UnsupportedOperationException`` is thrown.
Otherwise, if the server implementation throws an exception, that exception is serialised and rethrown on the client
side as if it was thrown from inside the called RPC method. These exceptions can be caught as normal.
Wire protocol
-------------
The client RPC wire protocol is not currently documented. To use it you must use the client library provided.
This is likely to change in a future release.
.. _CordaRPCClient: api/com.r3corda.client/-corda-r-p-c-client/index.html
.. _CordaRPCOps: api/com.r3corda.node.services.messaging/-corda-r-p-c-ops.html

View File

@ -39,13 +39,6 @@ Read on to learn:
node-administration node-administration
corda-configuration-files corda-configuration-files
.. toctree::
:maxdepth: 2
:caption: Contracts
contract-catalogue
contract-irs
.. toctree:: .. toctree::
:maxdepth: 2 :maxdepth: 2
:caption: Tutorials :caption: Tutorials
@ -59,6 +52,19 @@ Read on to learn:
event-scheduling event-scheduling
secure-coding-guidelines secure-coding-guidelines
.. toctree::
:maxdepth: 2
:caption: Contracts
contract-catalogue
contract-irs
.. toctree::
:maxdepth: 2
:caption: Node API
clientrpc
.. toctree:: .. toctree::
:maxdepth: 2 :maxdepth: 2
:caption: Appendix :caption: Appendix

View File

@ -64,6 +64,7 @@ class Node(dir: Path, val p2pAddr: HostAndPort, val webServerAddr: HostAndPort,
val messagingServerAddr: HostAndPort? = null) : AbstractNode(dir, configuration, networkMapAddress, advertisedServices, clock) { val messagingServerAddr: HostAndPort? = null) : AbstractNode(dir, configuration, networkMapAddress, advertisedServices, clock) {
companion object { companion object {
/** The port that is used by default if none is specified. As you know, 31337 is the most elite number. */ /** The port that is used by default if none is specified. As you know, 31337 is the most elite number. */
@JvmField
val DEFAULT_PORT = 31337 val DEFAULT_PORT = 31337
} }
@ -122,10 +123,11 @@ class Node(dir: Path, val p2pAddr: HostAndPort, val webServerAddr: HostAndPort,
messageBroker = ArtemisMessagingServer(dir, configuration, p2pAddr, services.networkMapCache) messageBroker = ArtemisMessagingServer(dir, configuration, p2pAddr, services.networkMapCache)
p2pAddr p2pAddr
}() }()
val ops = ServerRPCOps(services)
if (networkMapService != null) { if (networkMapService != null) {
return NodeMessagingClient(dir, configuration, serverAddr, services.storageService.myLegalIdentityKey.public, serverThread) return NodeMessagingClient(dir, configuration, serverAddr, services.storageService.myLegalIdentityKey.public, serverThread, rpcOps = ops)
} else { } else {
return NodeMessagingClient(dir, configuration, serverAddr, null, serverThread) return NodeMessagingClient(dir, configuration, serverAddr, null, serverThread, rpcOps = ops)
} }
} }

View File

@ -0,0 +1,14 @@
package com.r3corda.node.internal
import com.r3corda.node.services.api.ServiceHubInternal
import com.r3corda.node.services.messaging.CordaRPCOps
/**
* Server side implementations of RPCs available to MQ based client tools. Execution takes place on the server
* thread (i.e. serially). Arguments are serialised and deserialised automatically.
*/
class ServerRPCOps(services: ServiceHubInternal) : CordaRPCOps {
override val protocolVersion: Int = 0
// TODO: Add useful RPCs for client apps (examining the vault, etc)
}

View File

@ -8,7 +8,7 @@ import com.r3corda.core.node.services.TxWritableStorageService
import com.r3corda.core.protocols.ProtocolLogic import com.r3corda.core.protocols.ProtocolLogic
import com.r3corda.core.protocols.ProtocolLogicRefFactory import com.r3corda.core.protocols.ProtocolLogicRefFactory
interface MessagingServiceInternal: MessagingService { interface MessagingServiceInternal : MessagingService {
/** /**
* Initiates shutdown: if called from a thread that isn't controlled by the executor passed to the constructor * Initiates shutdown: if called from a thread that isn't controlled by the executor passed to the constructor
* then this will block until all in-flight messages have finished being handled and acknowledged. If called * then this will block until all in-flight messages have finished being handled and acknowledged. If called

View File

@ -1,10 +1,7 @@
package com.r3corda.node.services.config package com.r3corda.node.services.config
import com.google.common.net.HostAndPort import com.google.common.net.HostAndPort
import com.r3corda.core.crypto.Party
import com.r3corda.core.crypto.generateKeyPair
import com.r3corda.core.messaging.SingleMessageRecipient import com.r3corda.core.messaging.SingleMessageRecipient
import com.r3corda.core.node.NodeInfo
import com.r3corda.core.node.services.ServiceType import com.r3corda.core.node.services.ServiceType
import com.r3corda.node.internal.Node import com.r3corda.node.internal.Node
import com.r3corda.node.serialization.NodeClock import com.r3corda.node.serialization.NodeClock
@ -25,12 +22,17 @@ import java.util.*
import kotlin.reflect.KProperty import kotlin.reflect.KProperty
import kotlin.reflect.jvm.javaType import kotlin.reflect.jvm.javaType
interface NodeConfiguration { interface NodeSSLConfiguration {
val keyStorePassword: String
val trustStorePassword: String
// TODO: Move cert paths into this interface as well.
}
interface NodeConfiguration : NodeSSLConfiguration {
val myLegalName: String val myLegalName: String
val exportJMXto: String val exportJMXto: String
val nearestCity: String val nearestCity: String
val keyStorePassword: String
val trustStorePassword: String
val dataSourceProperties: Properties get() = Properties() val dataSourceProperties: Properties get() = Properties()
companion object { companion object {

View File

@ -5,10 +5,12 @@ import com.google.common.net.HostAndPort
import com.r3corda.core.crypto.X509Utilities import com.r3corda.core.crypto.X509Utilities
import com.r3corda.core.crypto.parsePublicKeyBase58 import com.r3corda.core.crypto.parsePublicKeyBase58
import com.r3corda.core.crypto.toBase58String import com.r3corda.core.crypto.toBase58String
import com.r3corda.core.div
import com.r3corda.core.messaging.MessageRecipients import com.r3corda.core.messaging.MessageRecipients
import com.r3corda.core.messaging.SingleMessageRecipient import com.r3corda.core.messaging.SingleMessageRecipient
import com.r3corda.core.serialization.SingletonSerializeAsToken import com.r3corda.core.serialization.SingletonSerializeAsToken
import com.r3corda.node.services.config.NodeConfiguration import com.r3corda.core.use
import com.r3corda.node.services.config.NodeSSLConfiguration
import org.apache.activemq.artemis.api.core.SimpleString import org.apache.activemq.artemis.api.core.SimpleString
import org.apache.activemq.artemis.api.core.TransportConfiguration import org.apache.activemq.artemis.api.core.TransportConfiguration
import org.apache.activemq.artemis.core.remoting.impl.netty.NettyAcceptorFactory import org.apache.activemq.artemis.core.remoting.impl.netty.NettyAcceptorFactory
@ -16,21 +18,27 @@ import org.apache.activemq.artemis.core.remoting.impl.netty.NettyConnectorFactor
import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants
import java.nio.file.Files import java.nio.file.Files
import java.nio.file.Path import java.nio.file.Path
import java.security.KeyStore
import java.security.PublicKey import java.security.PublicKey
/** /**
* The base class for Artemis services that defines shared data structures and transport configuration * The base class for Artemis services that defines shared data structures and transport configuration
* *
* @param directory A place where Artemis can stash its message journal and other files. * @param certificatePath A place where Artemis can stash its message journal and other files.
* @param config The config object is used to pass in the passwords for the certificate KeyStore and TrustStore * @param config The config object is used to pass in the passwords for the certificate KeyStore and TrustStore
*/ */
abstract class ArtemisMessagingComponent(val directory: Path, val config: NodeConfiguration) : SingletonSerializeAsToken() { abstract class ArtemisMessagingComponent(val certificatePath: Path, val config: NodeSSLConfiguration) : SingletonSerializeAsToken() {
private val keyStorePath = directory.resolve("certificates").resolve("sslkeystore.jks") val keyStorePath: Path = certificatePath / "sslkeystore.jks"
private val trustStorePath = directory.resolve("certificates").resolve("truststore.jks") val trustStorePath: Path = certificatePath / "truststore.jks"
companion object { companion object {
init {
System.setProperty("org.jboss.logging.provider", "slf4j")
}
const val PEERS_PREFIX = "peers." const val PEERS_PREFIX = "peers."
const val CLIENTS_PREFIX = "clients."
const val RPC_REQUESTS_QUEUE = "rpc.requests"
@JvmStatic @JvmStatic
protected val NETWORK_MAP_ADDRESS = SimpleString(PEERS_PREFIX +"networkmap") protected val NETWORK_MAP_ADDRESS = SimpleString(PEERS_PREFIX +"networkmap")
@ -70,8 +78,12 @@ abstract class ArtemisMessagingComponent(val directory: Path, val config: NodeCo
override val queueName: SimpleString = NETWORK_MAP_ADDRESS override val queueName: SimpleString = NETWORK_MAP_ADDRESS
} }
// In future: can contain onion routing info, etc. /**
protected data class NodeAddress(val identity: PublicKey, override val hostAndPort: HostAndPort) : SingleMessageRecipient, ArtemisAddress { * This is the class used to implement [SingleMessageRecipient], for now. Note that in future this class
* may change or evolve and code that relies upon it being a simple host/port may not function correctly.
* For instance it may contain onion routing data.
*/
data class NodeAddress(val identity: PublicKey, override val hostAndPort: HostAndPort) : SingleMessageRecipient, ArtemisAddress {
override val queueName: SimpleString by lazy { SimpleString(PEERS_PREFIX+identity.toBase58String()) } override val queueName: SimpleString by lazy { SimpleString(PEERS_PREFIX+identity.toBase58String()) }
override fun toString(): String { override fun toString(): String {
@ -79,18 +91,9 @@ abstract class ArtemisMessagingComponent(val directory: Path, val config: NodeCo
} }
} }
protected fun tryParseKeyFromQueueName(queueName: SimpleString): PublicKey? { protected fun parseKeyFromQueueName(name: String): PublicKey {
val name = queueName.toString() require(name.startsWith(PEERS_PREFIX))
if(!name.startsWith(PEERS_PREFIX)) { return parsePublicKeyBase58(name.substring(PEERS_PREFIX.length))
return null
}
val keyCode = name.substring(PEERS_PREFIX.length)
return try {
parsePublicKeyBase58(keyCode)
} catch (ex: Exception) {
null
}
} }
protected enum class ConnectionDirection { INBOUND, OUTBOUND } protected enum class ConnectionDirection { INBOUND, OUTBOUND }
@ -105,7 +108,21 @@ abstract class ArtemisMessagingComponent(val directory: Path, val config: NodeCo
"TLS_ECDH_ECDSA_WITH_AES_128_GCM_SHA256", "TLS_ECDH_ECDSA_WITH_AES_128_GCM_SHA256",
"TLS_ECDH_RSA_WITH_AES_128_GCM_SHA256", "TLS_ECDH_RSA_WITH_AES_128_GCM_SHA256",
"TLS_DHE_RSA_WITH_AES_128_GCM_SHA256", "TLS_DHE_RSA_WITH_AES_128_GCM_SHA256",
"TLS_DHE_DSS_WITH_AES_128_GCM_SHA256") "TLS_DHE_DSS_WITH_AES_128_GCM_SHA256"
)
/**
* Returns nothing if the keystore was opened OK or throws if not. Useful to check the password, as
* unfortunately Artemis tends to bury the exception when the password is wrong.
*/
fun checkStorePasswords() {
keyStorePath.use {
KeyStore.getInstance("JKS").load(it, config.keyStorePassword.toCharArray())
}
trustStorePath.use {
KeyStore.getInstance("JKS").load(it, config.trustStorePassword.toCharArray())
}
}
protected fun tcpTransport(direction: ConnectionDirection, host: String, port: Int) = protected fun tcpTransport(direction: ConnectionDirection, host: String, port: Int) =
TransportConfiguration( TransportConfiguration(
@ -144,11 +161,7 @@ abstract class ArtemisMessagingComponent(val directory: Path, val config: NodeCo
* the CA certs in Node resources. Then provision KeyStores into certificates folder under node path. * the CA certs in Node resources. Then provision KeyStores into certificates folder under node path.
*/ */
fun configureWithDevSSLCertificate() { fun configureWithDevSSLCertificate() {
Files.createDirectories(certificatePath)
val keyStorePath = directory.resolve("certificates").resolve("sslkeystore.jks")
val trustStorePath = directory.resolve("certificates").resolve("truststore.jks")
Files.createDirectories(directory.resolve("certificates"))
if (!Files.exists(trustStorePath)) { if (!Files.exists(trustStorePath)) {
Files.copy(javaClass.classLoader.getResourceAsStream("com/r3corda/node/internal/certificates/cordatruststore.jks"), Files.copy(javaClass.classLoader.getResourceAsStream("com/r3corda/node/internal/certificates/cordatruststore.jks"),
trustStorePath) trustStorePath)

View File

@ -2,6 +2,7 @@ package com.r3corda.node.services.messaging
import com.google.common.net.HostAndPort import com.google.common.net.HostAndPort
import com.r3corda.core.ThreadBox import com.r3corda.core.ThreadBox
import com.r3corda.core.crypto.AddressFormatException
import com.r3corda.core.crypto.newSecureRandom import com.r3corda.core.crypto.newSecureRandom
import com.r3corda.core.messaging.SingleMessageRecipient import com.r3corda.core.messaging.SingleMessageRecipient
import com.r3corda.core.node.NodeInfo import com.r3corda.core.node.NodeInfo
@ -117,7 +118,7 @@ class ArtemisMessagingServer(directory: Path,
} }
private fun configureAndStartServer() { private fun configureAndStartServer() {
val config = createArtemisConfig(directory, myHostPort).apply { val config = createArtemisConfig(certificatePath, myHostPort).apply {
securityRoles = mapOf( securityRoles = mapOf(
"#" to setOf(Role("internal", true, true, true, true, true, true, true)) "#" to setOf(Role("internal", true, true, true, true, true, true, true))
) )
@ -128,19 +129,32 @@ class ArtemisMessagingServer(directory: Path,
activeMQServer = ActiveMQServerImpl(config, securityManager).apply { activeMQServer = ActiveMQServerImpl(config, securityManager).apply {
// Throw any exceptions which are detected during startup // Throw any exceptions which are detected during startup
registerActivationFailureListener { exception -> throw exception } registerActivationFailureListener { exception -> throw exception }
// Deploy bridge for a newly created queue
// Some types of queue might need special preparation on our side, like dialling back or preparing
// a lazily initialised subsystem.
registerPostQueueCreationCallback { queueName -> registerPostQueueCreationCallback { queueName ->
log.info("Queue created: $queueName") log.debug("Queue created: $queueName")
if (queueName != NETWORK_MAP_ADDRESS) { if (queueName.startsWith(PEERS_PREFIX) && queueName != NETWORK_MAP_ADDRESS) {
val identity = tryParseKeyFromQueueName(queueName) try {
if (identity != null) { val identity = parseKeyFromQueueName(queueName.toString())
val nodeInfo = networkMapCache.getNodeByPublicKey(identity) val nodeInfo = networkMapCache.getNodeByPublicKey(identity)
if (nodeInfo != null) { if (nodeInfo != null) {
maybeDeployBridgeForAddress(queueName, nodeInfo.address) maybeDeployBridgeForAddress(queueName, nodeInfo.address)
} else {
log.error("Queue created for a peer that we don't know from the network map: $queueName")
} }
} catch (e: AddressFormatException) {
log.error("Protocol violation: Could not parse queue name as Base 58: $queueName")
} }
} }
} }
registerPostQueueDeletionCallback { address, qName ->
if (qName == address)
log.debug("Queue deleted: $qName")
else
log.debug("Queue deleted: $qName for $address")
}
} }
activeMQServer.start() activeMQServer.start()
} }
@ -148,7 +162,6 @@ class ArtemisMessagingServer(directory: Path,
private fun createArtemisConfig(directory: Path, hp: HostAndPort): Configuration { private fun createArtemisConfig(directory: Path, hp: HostAndPort): Configuration {
val config = ConfigurationImpl() val config = ConfigurationImpl()
setConfigDirectories(config, directory) setConfigDirectories(config, directory)
// We will be talking to our server purely in memory.
config.acceptorConfigurations = setOf( config.acceptorConfigurations = setOf(
tcpTransport(ConnectionDirection.INBOUND, "0.0.0.0", hp.port) tcpTransport(ConnectionDirection.INBOUND, "0.0.0.0", hp.port)
) )
@ -166,9 +179,9 @@ class ArtemisMessagingServer(directory: Path,
return ActiveMQJAASSecurityManager(InVMLoginModule::class.java.name, securityConfig) return ActiveMQJAASSecurityManager(InVMLoginModule::class.java.name, securityConfig)
} }
fun connectorExists(hostAndPort: HostAndPort) = hostAndPort.toString() in activeMQServer.configuration.connectorConfigurations private fun connectorExists(hostAndPort: HostAndPort) = hostAndPort.toString() in activeMQServer.configuration.connectorConfigurations
fun addConnector(hostAndPort: HostAndPort) = activeMQServer.configuration.addConnectorConfiguration( private fun addConnector(hostAndPort: HostAndPort) = activeMQServer.configuration.addConnectorConfiguration(
hostAndPort.toString(), hostAndPort.toString(),
tcpTransport( tcpTransport(
ConnectionDirection.OUTBOUND, ConnectionDirection.OUTBOUND,
@ -177,37 +190,32 @@ class ArtemisMessagingServer(directory: Path,
) )
) )
fun bridgeExists(name: SimpleString) = activeMQServer.clusterManager.bridges.containsKey(name.toString()) private fun bridgeExists(name: SimpleString) = activeMQServer.clusterManager.bridges.containsKey(name.toString())
fun deployBridge(hostAndPort: HostAndPort, name: SimpleString) = activeMQServer.deployBridge(BridgeConfiguration().apply { private fun deployBridge(hostAndPort: HostAndPort, name: SimpleString) {
val nameStr = name.toString() activeMQServer.deployBridge(BridgeConfiguration().apply {
setName(nameStr) val nameStr = name.toString()
queueName = nameStr setName(nameStr)
forwardingAddress = nameStr queueName = nameStr
staticConnectors = listOf(hostAndPort.toString()) forwardingAddress = nameStr
confirmationWindowSize = 100000 // a guess staticConnectors = listOf(hostAndPort.toString())
}) confirmationWindowSize = 100000 // a guess
})
}
/** /**
* For every queue created we need to have a bridge deployed in case the address of the queue * For every queue created we need to have a bridge deployed in case the address of the queue
* is that of a remote party * is that of a remote party.
*/ */
private fun maybeDeployBridgeForAddress(name: SimpleString, address: SingleMessageRecipient) { private fun maybeDeployBridgeForAddress(name: SimpleString, nodeInfo: SingleMessageRecipient) {
val hostAndPort = toHostAndPort(address) require(name.startsWith(PEERS_PREFIX))
val hostAndPort = toHostAndPort(nodeInfo)
if (hostAndPort == myHostPort) { if (hostAndPort == myHostPort)
return return
} if (!connectorExists(hostAndPort))
if (!connectorExists(hostAndPort)) {
log.info("add connector $hostAndPort")
addConnector(hostAndPort) addConnector(hostAndPort)
} if (!bridgeExists(name))
if (!bridgeExists(name)) {
log.info("add bridge $hostAndPort $name")
deployBridge(hostAndPort, name) deployBridge(hostAndPort, name)
}
} }
private fun maybeDestroyBridge(name: SimpleString) { private fun maybeDestroyBridge(name: SimpleString) {

View File

@ -0,0 +1,11 @@
package com.r3corda.node.services.messaging
import rx.Observable
/**
* RPC operations that the node exposes to clients using the Java client library. These can be called from
* client apps and are implemented by the node in the [ServerRPCOps] class.
*/
interface CordaRPCOps : RPCOps {
// TODO: Add useful RPCs for client apps (examining the vault, etc)
}

View File

@ -2,7 +2,9 @@ package com.r3corda.node.services.messaging
import com.google.common.net.HostAndPort import com.google.common.net.HostAndPort
import com.r3corda.core.ThreadBox import com.r3corda.core.ThreadBox
import com.r3corda.core.div
import com.r3corda.core.messaging.* import com.r3corda.core.messaging.*
import com.r3corda.core.serialization.SerializedBytes
import com.r3corda.core.serialization.opaque import com.r3corda.core.serialization.opaque
import com.r3corda.core.utilities.loggerFor import com.r3corda.core.utilities.loggerFor
import com.r3corda.node.services.api.MessagingServiceInternal import com.r3corda.node.services.api.MessagingServiceInternal
@ -21,6 +23,8 @@ import java.util.concurrent.CountDownLatch
import java.util.concurrent.Executor import java.util.concurrent.Executor
import javax.annotation.concurrent.ThreadSafe import javax.annotation.concurrent.ThreadSafe
// TODO: Stop the wallet explorer and other clients from using this class and get rid of persistentInbox
/** /**
* This class implements the [MessagingService] API using Apache Artemis, the successor to their ActiveMQ product. * This class implements the [MessagingService] API using Apache Artemis, the successor to their ActiveMQ product.
* Artemis is a message queue broker and here we run a client connecting to the specified broker instance * Artemis is a message queue broker and here we run a client connecting to the specified broker instance
@ -30,6 +34,10 @@ import javax.annotation.concurrent.ThreadSafe
* are blocked until the handler is scheduled and completed. This allows backpressure to propagate from the given * are blocked until the handler is scheduled and completed. This allows backpressure to propagate from the given
* executor through into Artemis and from there, back through to senders. * executor through into Artemis and from there, back through to senders.
* *
* An implementation of [CordaRPCOps] can be provided. If given, clients using the CordaMQClient RPC library can
* invoke methods on the provided implementation. There is more documentation on this in the docsite and the
* CordaRPCClient class.
*
* @param serverHostPort The address of the broker instance to connect to (might be running in the same process) * @param serverHostPort The address of the broker instance to connect to (might be running in the same process)
* @param myIdentity Either the public key to be used as the ArtemisMQ address and queue name for the node globally, or null to indicate * @param myIdentity Either the public key to be used as the ArtemisMQ address and queue name for the node globally, or null to indicate
* that this is a NetworkMapService node which will be bound globally to the name "networkmap" * that this is a NetworkMapService node which will be bound globally to the name "networkmap"
@ -43,7 +51,9 @@ class NodeMessagingClient(directory: Path,
val serverHostPort: HostAndPort, val serverHostPort: HostAndPort,
val myIdentity: PublicKey?, val myIdentity: PublicKey?,
val executor: AffinityExecutor, val executor: AffinityExecutor,
val persistentInbox: Boolean = true) : ArtemisMessagingComponent(directory, config), MessagingServiceInternal { val persistentInbox: Boolean = true,
private val rpcOps: CordaRPCOps? = null)
: ArtemisMessagingComponent(directory / "certificates", config), MessagingServiceInternal {
companion object { companion object {
val log = loggerFor<NodeMessagingClient>() val log = loggerFor<NodeMessagingClient>()
@ -68,9 +78,12 @@ class NodeMessagingClient(directory: Path,
var running = false var running = false
val knownQueues = mutableSetOf<SimpleString>() val knownQueues = mutableSetOf<SimpleString>()
var producer: ClientProducer? = null var producer: ClientProducer? = null
var consumer: ClientConsumer? = null var p2pConsumer: ClientConsumer? = null
var session: ClientSession? = null var session: ClientSession? = null
var clientFactory: ClientSessionFactory? = null var clientFactory: ClientSessionFactory? = null
// Consumer for inbound client RPC messages.
var rpcConsumer: ClientConsumer? = null
var rpcNotificationConsumer: ClientConsumer? = null
// TODO: This is not robust and needs to be replaced by more intelligently using the message queue server. // TODO: This is not robust and needs to be replaced by more intelligently using the message queue server.
var undeliveredMessages = listOf<Message>() var undeliveredMessages = listOf<Message>()
@ -99,7 +112,7 @@ class NodeMessagingClient(directory: Path,
started = true started = true
log.info("Connecting to server: $serverHostPort") log.info("Connecting to server: $serverHostPort")
// Connect to our server. // Connect to our server. TODO: This should use the in-VM transport.
val tcpTransport = tcpTransport(ConnectionDirection.OUTBOUND, serverHostPort.hostText, serverHostPort.port) val tcpTransport = tcpTransport(ConnectionDirection.OUTBOUND, serverHostPort.hostText, serverHostPort.port)
val locator = ActiveMQClient.createServerLocatorWithoutHA(tcpTransport) val locator = ActiveMQClient.createServerLocatorWithoutHA(tcpTransport)
clientFactory = locator.createSessionFactory() clientFactory = locator.createSessionFactory()
@ -107,30 +120,43 @@ class NodeMessagingClient(directory: Path,
// Create a session and configure to commit manually after each acknowledge. (N.B. ackBatchSize is in Bytes!!!) // Create a session and configure to commit manually after each acknowledge. (N.B. ackBatchSize is in Bytes!!!)
val session = clientFactory!!.createSession(true, true, 1) val session = clientFactory!!.createSession(true, true, 1)
this.session = session this.session = session
session.start()
// Create a queue on which to receive messages and set up the handler. // Create a general purpose producer.
producer = session.createProducer()
// Create a queue, consumer and producer for handling P2P network messages.
val queueName = toQueueName(myAddress) val queueName = toQueueName(myAddress)
val query = session.queueQuery(queueName) val query = session.queueQuery(queueName)
if (!query.isExists) { if (!query.isExists) {
session.createQueue(queueName, queueName, persistentInbox) session.createQueue(queueName, queueName, persistentInbox)
} }
knownQueues.add(queueName) knownQueues.add(queueName)
consumer = session.createConsumer(queueName) p2pConsumer = session.createConsumer(queueName)
producer = session.createProducer()
session.start() // Create an RPC queue and consumer: this will service locally connected clients only (not via a
// bridge) and those clients must have authenticated. We could use a single consumer for everything
// and perhaps we should, but these queues are not worth persisting.
if (rpcOps != null) {
session.createTemporaryQueue(RPC_REQUESTS_QUEUE, RPC_REQUESTS_QUEUE)
session.createTemporaryQueue("activemq.notifications", "rpc.qremovals", "_AMQ_NotifType = 1")
rpcConsumer = session.createConsumer(RPC_REQUESTS_QUEUE)
rpcNotificationConsumer = session.createConsumer("rpc.qremovals")
}
} }
} }
private var shutdownLatch = CountDownLatch(1) private var shutdownLatch = CountDownLatch(1)
/** Starts the event loop: this method only returns once [stop] has been called. */ /** Starts the p2p event loop: this method only returns once [stop] has been called. */
fun run() { fun run() {
val consumer = state.locked { val consumer = state.locked {
check(started) check(started)
check(!running) { "run can't be called twice" } check(!running) { "run can't be called twice" }
running = true running = true
consumer!! // Optionally, start RPC dispatch.
dispatcher?.start(rpcConsumer!!, rpcNotificationConsumer!!, executor)
p2pConsumer!!
} }
while (true) { while (true) {
@ -254,13 +280,13 @@ class NodeMessagingClient(directory: Path,
// We allow stop() to be called without a run() in between, but it must have at least been started. // We allow stop() to be called without a run() in between, but it must have at least been started.
check(started) check(started)
val c = consumer ?: throw IllegalStateException("stop can't be called twice") val c = p2pConsumer ?: throw IllegalStateException("stop can't be called twice")
try { try {
c.close() c.close()
} catch(e: ActiveMQObjectClosedException) { } catch(e: ActiveMQObjectClosedException) {
// Ignore it: this can happen if the server has gone away before we do. // Ignore it: this can happen if the server has gone away before we do.
} }
consumer = null p2pConsumer = null
val prevRunning = running val prevRunning = running
running = false running = false
prevRunning prevRunning
@ -272,6 +298,10 @@ class NodeMessagingClient(directory: Path,
// Only first caller to gets running true to protect against double stop, which seems to happen in some integration tests. // Only first caller to gets running true to protect against double stop, which seems to happen in some integration tests.
if (running) { if (running) {
state.locked { state.locked {
rpcConsumer?.close()
rpcConsumer = null
rpcNotificationConsumer?.close()
rpcNotificationConsumer = null
producer?.close() producer?.close()
producer = null producer = null
// Ensure any trailing messages are committed to the journal // Ensure any trailing messages are committed to the journal
@ -305,7 +335,7 @@ class NodeMessagingClient(directory: Path,
state.alreadyLocked { state.alreadyLocked {
val queueQuery = session!!.queueQuery(queueName) val queueQuery = session!!.queueQuery(queueName)
if (!queueQuery.isExists) { if (!queueQuery.isExists) {
log.info("create client queue $queueName") log.info("Create fresh queue $queueName")
session!!.createQueue(queueName, queueName, true /* durable */) session!!.createQueue(queueName, queueName, true /* durable */)
} }
} }
@ -346,6 +376,17 @@ class NodeMessagingClient(directory: Path,
} }
} }
override fun createMessage(topic: String, sessionID: Long, data: ByteArray): Message override fun createMessage(topic: String, sessionID: Long, data: ByteArray) = createMessage(TopicSession(topic, sessionID), data)
= createMessage(TopicSession(topic, sessionID), data)
private fun createRPCDispatcher(ops: CordaRPCOps) = object : RPCDispatcher(ops) {
override fun send(bits: SerializedBytes<*>, toAddress: String) {
state.locked {
val msg = session!!.createMessage(false)
msg.writeBodyBufferBytes(bits.bits)
producer!!.send(toAddress, msg)
}
}
}
private val dispatcher = if (rpcOps != null) createRPCDispatcher(rpcOps) else null
} }

View File

@ -0,0 +1,129 @@
package com.r3corda.node.services.messaging
import com.esotericsoftware.kryo.Kryo
import com.esotericsoftware.kryo.KryoException
import com.esotericsoftware.kryo.Serializer
import com.esotericsoftware.kryo.io.Input
import com.esotericsoftware.kryo.io.Output
import com.google.common.collect.HashMultimap
import com.r3corda.core.ErrorOr
import com.r3corda.core.serialization.SerializedBytes
import com.r3corda.core.serialization.deserialize
import com.r3corda.core.serialization.serialize
import com.r3corda.core.utilities.debug
import com.r3corda.node.utilities.AffinityExecutor
import org.apache.activemq.artemis.api.core.client.ClientConsumer
import rx.Notification
import rx.Observable
import rx.Subscription
import java.lang.reflect.InvocationTargetException
import java.util.concurrent.atomic.AtomicInteger
// TODO: Exposing the authenticated message sender.
/**
* Intended to service transient clients only (not p2p nodes) for short-lived, transient request/response pairs.
* If you need robustness, this is the wrong system. If you don't want a response, this is probably the
* wrong system (you could just send a message). If you want complex customisation of how requests/responses
* are handled, this is probably the wrong system.
*/
abstract class RPCDispatcher(val target: Any) {
private val methodTable = target.javaClass.declaredMethods.associateBy { it.name }
private val queueToSubscription = HashMultimap.create<String, Subscription>()
// Created afresh for every RPC that is annotated as returning observables. Every time an observable is
// encountered either in the RPC response or in an object graph that is being emitted by one of those
// observables, the handle counter is incremented and the server-side observable is subscribed to. The
// materialized observations are then sent to the queue the client created where they can be picked up.
//
// When the observables are deserialised on the client side, the handle is read from the byte stream and
// the queue is filtered to extract just those observations.
private inner class ObservableSerializer(private val toQName: String) : Serializer<Observable<Any>>() {
private val handleCounter = AtomicInteger()
override fun read(kryo: Kryo, input: Input, type: Class<Observable<Any>>): Observable<Any> {
throw UnsupportedOperationException("not implemented")
}
override fun write(kryo: Kryo, output: Output, obj: Observable<Any>) {
val handle = handleCounter.andIncrement
output.writeInt(handle, true)
// Observables can do three kinds of callback: "next" with a content object, "completed" and "error".
// Materializing the observable converts these three kinds of callback into a single stream of objects
// representing what happened, which is useful for us to send over the wire.
val subscription = obj.materialize().subscribe { materialised: Notification<out Any> ->
val newKryo = createRPCKryo(observableSerializer = this@ObservableSerializer)
val bits = MarshalledObservation(handle, materialised).serialize(newKryo)
rpcLog.debug("RPC sending observation: $materialised")
send(bits, toQName)
}
synchronized(queueToSubscription) {
queueToSubscription.put(toQName, subscription)
}
}
}
fun dispatch(msg: ClientRPCRequestMessage) {
val (argBytes, replyTo, observationsTo, name) = msg
val maybeArgs = argBytes.deserialize<Array<Any>>()
rpcLog.debug { "-> RPC -> $name(${maybeArgs.joinToString()}) [reply to $replyTo]" }
val response: ErrorOr<Any?> = ErrorOr.catch {
val method = methodTable[name] ?: throw RPCException("Received RPC for unknown method $name - possible client/server version skew?")
if (method.isAnnotationPresent(RPCReturnsObservables::class.java) && observationsTo == null)
throw RPCException("Received RPC without any destination for observations, but the RPC returns observables")
try {
method.invoke(target, *maybeArgs)
} catch (e: InvocationTargetException) {
throw e.cause!!
}
}
rpcLog.debug { "<- RPC <- $name = $response " }
val kryo = createRPCKryo(observableSerializer = if (observationsTo != null) ObservableSerializer(observationsTo) else null)
// Serialise, or send back a simple serialised ErrorOr structure if we couldn't do it.
val responseBits = try {
response.serialize(kryo)
} catch (e: KryoException) {
rpcLog.error("Failed to respond to inbound RPC $name", e)
ErrorOr.of(e).serialize(kryo)
}
send(responseBits, replyTo)
}
abstract fun send(bits: SerializedBytes<*>, toAddress: String)
fun start(rpcConsumer: ClientConsumer, rpcNotificationConsumer: ClientConsumer?, onExecutor: AffinityExecutor) {
rpcNotificationConsumer?.setMessageHandler { msg ->
val qName = msg.getStringProperty("_AMQ_RoutingName")
val subscriptions = synchronized(queueToSubscription) {
queueToSubscription.removeAll(qName)
}
if (subscriptions.isNotEmpty()) {
rpcLog.debug("Observable queue was deleted, unsubscribing: $qName")
subscriptions.forEach { it.unsubscribe() }
}
}
rpcConsumer.setMessageHandler { msg ->
msg.acknowledge()
// All RPCs run on the main server thread, in order to avoid running concurrently with
// potentially state changing requests from other nodes and each other. If we need to
// give better latency to client RPCs in future we could use an executor that supports
// job priorities.
onExecutor.execute {
try {
val rpcMessage = msg.toRPCRequestMessage()
dispatch(rpcMessage)
} catch(e: RPCException) {
rpcLog.warn("Received malformed client RPC message: ${e.message}")
rpcLog.trace("RPC exception", e)
} catch(e: Throwable) {
rpcLog.error("Uncaught exception when dispatching client RPC", e)
}
}
}
}
}

View File

@ -0,0 +1,142 @@
package com.r3corda.node.services.messaging
import com.esotericsoftware.kryo.Kryo
import com.esotericsoftware.kryo.Registration
import com.esotericsoftware.kryo.Serializer
import com.esotericsoftware.kryo.io.Input
import com.esotericsoftware.kryo.io.Output
import com.r3corda.core.ErrorOr
import com.r3corda.core.crypto.Party
import com.r3corda.core.serialization.*
import com.r3corda.core.transactions.SignedTransaction
import com.r3corda.core.transactions.WireTransaction
import de.javakaffee.kryoserializers.ArraysAsListSerializer
import de.javakaffee.kryoserializers.guava.*
import org.apache.activemq.artemis.api.core.client.ClientMessage
import org.objenesis.strategy.StdInstantiatorStrategy
import org.slf4j.LoggerFactory
import rx.Notification
import rx.Observable
import java.time.Instant
import java.util.*
/** Global RPC logger */
val rpcLog by lazy { LoggerFactory.getLogger("com.r3corda.rpc") }
/** Used in the RPC wire protocol to wrap an observation with the handle of the observable it's intended for. */
data class MarshalledObservation(val forHandle: Int, val what: Notification<*>)
/**
* If an RPC is tagged with this annotation it may return one or more observables anywhere in its response graph.
* Calling such a method comes with consequences: it's slower, and consumes server side resources as observations
* will buffer up on the server until they're consumed by the client.
*/
@Target(AnnotationTarget.FUNCTION)
@MustBeDocumented
annotation class RPCReturnsObservables
/** Records the protocol version in which this RPC was added. */
@Target(AnnotationTarget.FUNCTION)
@MustBeDocumented
annotation class RPCSinceVersion(val version: Int)
/** The contents of an RPC request message, separated from the MQ layer. */
data class ClientRPCRequestMessage(
val args: SerializedBytes<Array<Any>>,
val replyToAddress: String,
val observationsToAddress: String?,
val methodName: String
) {
companion object {
const val REPLY_TO = "reply-to"
const val OBSERVATIONS_TO = "observations-to"
const val METHOD_NAME = "method-name"
}
}
/**
* Base interface that all RPC servers must implement. Note: in Corda there's only one RPC interface. This base
* interface is here in case we split the RPC system out into a separate library one day.
*/
interface RPCOps {
/** Returns the RPC protocol version. Exists since version 0 so guaranteed to be present. */
val protocolVersion: Int
}
/**
* Thrown to indicate a fatal error in the RPC system itself, as opposed to an error generated by the invoked
* method.
*/
open class RPCException(msg: String, cause: Throwable?) : RuntimeException(msg, cause) {
constructor(msg: String) : this(msg, null)
class DeadlineExceeded(rpcName: String) : RPCException("Deadline exceeded on call to $rpcName")
}
/** Convert an Artemis [ClientMessage] to a MQ-neutral [ClientRPCRequestMessage]. */
fun ClientMessage.toRPCRequestMessage(): ClientRPCRequestMessage {
fun ClientMessage.requiredString(name: String): String = getStringProperty(name) ?: throw RPCException("Malformed request message: missing $name property")
val methodName = requiredString(ClientRPCRequestMessage.METHOD_NAME)
// TODO: Look up the authenticated sender identity once we upgrade to Artemis 1.4 and use that instead.
// This current approach is insecure: one client could send an RPC with a reply-to address owned by
// another, although they'd have to be able to figure out the other client ID first.
// We also need that to figure out what RPCs are allowed.
val replyTo = requiredString(ClientRPCRequestMessage.REPLY_TO)
val observationsTo = getStringProperty(ClientRPCRequestMessage.OBSERVATIONS_TO)
val argBytes = ByteArray(bodySize).apply { bodyBuffer.readBytes(this) }
check(argBytes.isNotEmpty())
return ClientRPCRequestMessage(SerializedBytes(argBytes), replyTo, observationsTo, methodName)
}
// The Kryo used for the RPC wire protocol. Every type in the wire protocol is listed here explicitly.
// This is annoying to write out, but will make it easier to formalise the wire protocol when the time comes,
// because we can see everything we're using in one place.
private class RPCKryo(private val observableSerializer: Serializer<Observable<Any>>? = null) : Kryo() {
init {
isRegistrationRequired = true
// Allow construction of objects using a JVM backdoor that skips invoking the constructors, if there is no
// no-arg constructor available.
instantiatorStrategy = Kryo.DefaultInstantiatorStrategy(StdInstantiatorStrategy())
register(Arrays.asList("").javaClass, ArraysAsListSerializer())
register(Instant::class.java, ReferencesAwareJavaSerializer)
register(SignedTransaction::class.java, ImmutableClassSerializer(SignedTransaction::class))
register(WireTransaction::class.java, WireTransactionSerializer)
register(SerializedBytes::class.java, SerializedBytesSerializer)
register(Party::class.java)
ImmutableListSerializer.registerSerializers(this)
ImmutableSetSerializer.registerSerializers(this)
ImmutableSortedSetSerializer.registerSerializers(this)
ImmutableMapSerializer.registerSerializers(this)
ImmutableMultimapSerializer.registerSerializers(this)
noReferencesWithin<WireTransaction>()
register(ErrorOr::class.java)
register(MarshalledObservation::class.java, ImmutableClassSerializer(MarshalledObservation::class))
register(Notification::class.java)
register(Notification.Kind::class.java)
register(kotlin.Pair::class.java)
// Exceptions. We don't bother sending the stack traces as the client will fill in its own anyway.
register(IllegalArgumentException::class.java)
register(RPCException::class.java)
register(Array<StackTraceElement>::class.java, object : Serializer<Array<StackTraceElement>>() {
override fun read(kryo: Kryo, input: Input, type: Class<Array<StackTraceElement>>): Array<StackTraceElement> = emptyArray()
override fun write(kryo: Kryo, output: Output, `object`: Array<StackTraceElement>) {}
})
register(Collections.unmodifiableList(emptyList<String>()).javaClass)
}
val observableRegistration: Registration? = if (observableSerializer != null) register(Observable::class.java, observableSerializer) else null
override fun getRegistration(type: Class<*>): Registration {
if (Observable::class.java.isAssignableFrom(type))
return observableRegistration ?: throw IllegalStateException("This RPC was not annotated with @RPCReturnsObservables")
return super.getRegistration(type)
}
}
fun createRPCKryo(observableSerializer: Serializer<Observable<Any>>? = null): Kryo = RPCKryo(observableSerializer)

View File

@ -14,13 +14,13 @@ fi
echo "Generating docsite ..." echo "Generating docsite ..."
echo echo
( cd docs; make html ) ( cd docs; make clean html )
echo echo
echo "Generating API docs ..." echo "Generating API docs ..."
echo echo
java -jar lib/dokka.jar -output docs/build/html/api core/src/main/kotlin contracts/src/main/kotlin node/src/main/kotlin src/main/kotlin | grep -v "No documentation for" java -jar lib/dokka.jar -output docs/build/html/api core/src/main/kotlin contracts/src/main/kotlin node/src/main/kotlin src/main/kotlin client/src/main/kotlin | grep -v "No documentation for"
echo echo
echo "Writing robots.txt" echo "Writing robots.txt"

View File

@ -5,5 +5,4 @@ include 'core'
include 'node' include 'node'
include 'client' include 'client'
include 'experimental' include 'experimental'
include 'test-utils' include 'test-utils'