├── ripc-rxjava1 └── src │ ├── test │ └── java │ │ └── io │ │ └── ripc │ │ └── rx │ │ ├── package-info.java │ │ └── RxTcpServerTests.java │ └── main │ └── java │ └── io │ └── ripc │ └── rx │ ├── package-info.java │ └── protocol │ └── tcp │ ├── RxTcpInterceptor.java │ ├── RxTcpHandler.java │ ├── RxTcpServer.java │ └── RxConnection.java ├── .gitignore ├── ripc-protocol-tcp └── src │ ├── main │ └── java │ │ └── io │ │ └── ripc │ │ └── protocol │ │ └── tcp │ │ ├── package-info.java │ │ ├── TcpInterceptor.java │ │ ├── TcpHandler.java │ │ ├── TcpConnection.java │ │ └── TcpServer.java │ └── test │ └── java │ └── io │ └── ripc │ └── protocol │ └── tcp │ └── package-info.java ├── ripc-transport-netty4 └── src │ ├── test │ └── java │ │ └── io │ │ └── ripc │ │ └── transport │ │ └── netty4 │ │ ├── package-info.java │ │ └── TcpServerTests.java │ └── main │ └── java │ └── io │ └── ripc │ └── transport │ └── netty4 │ ├── package-info.java │ └── tcp │ ├── TcpConnectionImpl.java │ ├── Netty4TcpServer.java │ └── ChannelToConnectionBridge.java ├── gradle └── wrapper │ ├── gradle-wrapper.jar │ └── gradle-wrapper.properties ├── gradle.properties ├── ripc-reactor └── src │ ├── main │ └── java │ │ └── io │ │ └── ripc │ │ └── reactor │ │ └── protocol │ │ └── tcp │ │ ├── ReactorTcpHandler.java │ │ ├── ReactorTcpConnection.java │ │ └── ReactorTcpServer.java │ └── test │ └── java │ └── io │ └── ripc │ └── reactor │ └── ReactorTcpServerTests.java ├── settings.gradle ├── ripc-reactor-examples └── src │ └── main │ ├── resources │ └── logback.xml │ └── java │ └── io │ ├── ripc │ └── reactor │ │ └── protocol │ │ └── tcp │ │ ├── ReactorTcpServerSample.java │ │ └── CodecSample.java │ └── netty │ └── handler │ └── codec │ └── json │ └── JsonObjectDecoder.java ├── ripc-transport-netty4-examples └── src │ └── main │ ├── resources │ └── logback.xml │ └── java │ └── io │ ├── ripc │ └── transport │ │ └── netty4 │ │ └── tcp │ │ ├── TcpServerSample.java │ │ └── CodecSample.java │ └── netty │ └── handler │ └── codec │ └── json │ └── JsonObjectDecoder.java ├── ripc-core └── src │ ├── test │ └── resources │ │ └── logback.xml │ └── main │ └── java │ └── io │ └── ripc │ └── io │ └── Buffer.java ├── README.md ├── ripc-rxjava1-examples └── src │ └── main │ └── java │ └── io │ └── rpc │ └── rx │ └── protocol │ └── tcp │ └── RxTcpServerSample.java ├── ripc-test └── src │ └── main │ └── java │ └── io │ └── ripc │ └── test │ ├── Publishers.java │ ├── SocketTestUtils.java │ └── internal │ ├── SubscriberWithContext.java │ └── PublisherFactory.java ├── gradlew.bat ├── gradlew └── LICENSE.txt /ripc-rxjava1/src/test/java/io/ripc/rx/package-info.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | */ 4 | -------------------------------------------------------------------------------- /ripc-rxjava1/src/main/java/io/ripc/rx/package-info.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | */ 4 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .DS_Store 2 | .gradle 3 | .idea 4 | out 5 | build 6 | tmp 7 | *.log 8 | *.i* 9 | -------------------------------------------------------------------------------- /ripc-protocol-tcp/src/main/java/io/ripc/protocol/tcp/package-info.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | */ 4 | -------------------------------------------------------------------------------- /ripc-protocol-tcp/src/test/java/io/ripc/protocol/tcp/package-info.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | */ 4 | -------------------------------------------------------------------------------- /ripc-transport-netty4/src/test/java/io/ripc/transport/netty4/package-info.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | */ 4 | -------------------------------------------------------------------------------- /gradle/wrapper/gradle-wrapper.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/reactive-ipc/reactive-ipc-jvm/HEAD/gradle/wrapper/gradle-wrapper.jar -------------------------------------------------------------------------------- /ripc-transport-netty4/src/main/java/io/ripc/transport/netty4/package-info.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Reactive IPC Netty 4.x Implementation 3 | */ 4 | -------------------------------------------------------------------------------- /gradle.properties: -------------------------------------------------------------------------------- 1 | description = Reactive Interprocess Communication Libraries for the JVM 2 | group = io.reactive-ipc 3 | version = 0.1.0.BUILD-SNAPSHOT -------------------------------------------------------------------------------- /ripc-protocol-tcp/src/main/java/io/ripc/protocol/tcp/TcpInterceptor.java: -------------------------------------------------------------------------------- 1 | package io.ripc.protocol.tcp; 2 | 3 | public interface TcpInterceptor { 4 | 5 | TcpHandler intercept(TcpHandler handler); 6 | 7 | } 8 | -------------------------------------------------------------------------------- /ripc-rxjava1/src/main/java/io/ripc/rx/protocol/tcp/RxTcpInterceptor.java: -------------------------------------------------------------------------------- 1 | package io.ripc.rx.protocol.tcp; 2 | 3 | public interface RxTcpInterceptor { 4 | 5 | RxTcpHandler intercept(RxTcpHandler handler); 6 | } 7 | -------------------------------------------------------------------------------- /ripc-rxjava1/src/main/java/io/ripc/rx/protocol/tcp/RxTcpHandler.java: -------------------------------------------------------------------------------- 1 | package io.ripc.rx.protocol.tcp; 2 | 3 | import rx.Observable; 4 | 5 | public interface RxTcpHandler { 6 | 7 | Observable handle(RxConnection connection); 8 | } 9 | -------------------------------------------------------------------------------- /ripc-protocol-tcp/src/main/java/io/ripc/protocol/tcp/TcpHandler.java: -------------------------------------------------------------------------------- 1 | package io.ripc.protocol.tcp; 2 | 3 | import org.reactivestreams.Publisher; 4 | 5 | public interface TcpHandler { 6 | 7 | Publisher handle(TcpConnection connection); 8 | } 9 | -------------------------------------------------------------------------------- /gradle/wrapper/gradle-wrapper.properties: -------------------------------------------------------------------------------- 1 | #Wed Sep 25 09:49:11 CDT 2013 2 | distributionBase=GRADLE_USER_HOME 3 | distributionPath=wrapper/dists 4 | zipStoreBase=GRADLE_USER_HOME 5 | zipStorePath=wrapper/dists 6 | distributionUrl=http\://services.gradle.org/distributions/gradle-2.3-all.zip 7 | -------------------------------------------------------------------------------- /ripc-reactor/src/main/java/io/ripc/reactor/protocol/tcp/ReactorTcpHandler.java: -------------------------------------------------------------------------------- 1 | package io.ripc.reactor.protocol.tcp; 2 | 3 | import org.reactivestreams.Publisher; 4 | import reactor.fn.Function; 5 | 6 | public interface ReactorTcpHandler extends Function, Publisher> { 7 | 8 | } -------------------------------------------------------------------------------- /settings.gradle: -------------------------------------------------------------------------------- 1 | rootProject.name = 'reactive-ipc' 2 | 3 | include 'ripc-core', 4 | 'ripc-protocol-tcp', 5 | 'ripc-transport-netty4', 6 | 'ripc-transport-netty4-examples', 7 | 'ripc-reactor', 8 | 'ripc-rxjava1', 9 | 'ripc-reactor-examples', 10 | 'ripc-rxjava1-examples', 11 | 'ripc-test' 12 | -------------------------------------------------------------------------------- /ripc-reactor-examples/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | -------------------------------------------------------------------------------- /ripc-transport-netty4-examples/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | -------------------------------------------------------------------------------- /ripc-core/src/test/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | -------------------------------------------------------------------------------- /ripc-protocol-tcp/src/main/java/io/ripc/protocol/tcp/TcpConnection.java: -------------------------------------------------------------------------------- 1 | package io.ripc.protocol.tcp; 2 | 3 | import org.reactivestreams.Publisher; 4 | 5 | /** 6 | * An abstraction for a TCP connection. 7 | * 8 | * @param The type of objects read from this connection. 9 | * @param The type of objects written to this connection. 10 | */ 11 | public interface TcpConnection extends Publisher { 12 | 13 | /** 14 | * Writes the passed stream of {@code data} and returns the result as a {@link Publisher}. All items emitted by 15 | * this stream are flushed on completion of the stream. 16 | * 17 | * @param data Data stream to write. 18 | * 19 | * @return Result of write. 20 | */ 21 | Publisher write(Publisher data); 22 | 23 | } 24 | -------------------------------------------------------------------------------- /ripc-reactor/src/main/java/io/ripc/reactor/protocol/tcp/ReactorTcpConnection.java: -------------------------------------------------------------------------------- 1 | package io.ripc.reactor.protocol.tcp; 2 | 3 | import io.ripc.protocol.tcp.TcpConnection; 4 | import org.reactivestreams.Publisher; 5 | import org.reactivestreams.Subscriber; 6 | import reactor.rx.Stream; 7 | 8 | /** 9 | * Created by jbrisbin on 5/28/15. 10 | */ 11 | public class ReactorTcpConnection extends Stream { 12 | 13 | private final TcpConnection transport; 14 | 15 | public ReactorTcpConnection(TcpConnection transport) { 16 | this.transport = transport; 17 | } 18 | 19 | public Publisher writeWith(Publisher out) { 20 | return transport.write(out); 21 | } 22 | 23 | @Override 24 | public void subscribe(Subscriber sub) { 25 | transport.subscribe(sub); 26 | } 27 | 28 | } 29 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Reactive Inter-Process Communication Library 2 | 3 | The intent of this project is to bring together Reactive Streams, RxJava, Reactor and similar efforts on networking libraries to create a “networking kernel” that can be the foundation of reactive, stream oriented IO for clients and servers supporting UDP, TCP, HTTP/1, HTTP/2 and WebSockets and a mechanism for pluggable codecs on top of all transports. 4 | 5 | The desired timeline is BETA (1.0.rc1) by September 2015 and GA (1.0 Final) by end of 2015. 6 | 7 | This would result in 3 layers of modules: API, Transport and Core. 8 | 9 | ![screen shot 2015-03-06 at 4 06 51 pm](https://cloud.githubusercontent.com/assets/813492/6538742/06420452-c41b-11e4-9467-93d1ac26e25a.png) 10 | 11 | Discussion of goals and motiviations can be found in [Issue #1](https://github.com/reactive-ipc/reactive-ipc-jvm/issues/1). 12 | -------------------------------------------------------------------------------- /ripc-rxjava1-examples/src/main/java/io/rpc/rx/protocol/tcp/RxTcpServerSample.java: -------------------------------------------------------------------------------- 1 | package io.rpc.rx.protocol.tcp; 2 | 3 | import io.netty.buffer.ByteBuf; 4 | import io.netty.buffer.Unpooled; 5 | import io.ripc.protocol.tcp.TcpServer; 6 | import io.ripc.rx.protocol.tcp.RxTcpServer; 7 | import io.ripc.transport.netty4.tcp.Netty4TcpServer; 8 | import rx.Observable; 9 | 10 | import static java.nio.charset.Charset.*; 11 | 12 | public class RxTcpServerSample { 13 | 14 | public static void main(String[] args) throws InterruptedException { 15 | 16 | TcpServer transport = Netty4TcpServer.create(0); 17 | 18 | RxTcpServer.create(transport) 19 | .startAndAwait(connection -> connection.flatMap(bb -> { 20 | String msgStr = "Hello " + bb.toString(defaultCharset()); 21 | ByteBuf msg = Unpooled.buffer().writeBytes(msgStr.getBytes()); 22 | return connection.write(Observable.just(msg)); 23 | })); 24 | } 25 | } -------------------------------------------------------------------------------- /ripc-transport-netty4-examples/src/main/java/io/ripc/transport/netty4/tcp/TcpServerSample.java: -------------------------------------------------------------------------------- 1 | package io.ripc.transport.netty4.tcp; 2 | 3 | 4 | import io.netty.buffer.ByteBuf; 5 | import io.netty.buffer.Unpooled; 6 | import io.ripc.test.Publishers; 7 | 8 | import static java.nio.charset.Charset.*; 9 | import static rx.RxReactiveStreams.*; 10 | 11 | public class TcpServerSample { 12 | 13 | public static void main(String[] args) { 14 | Netty4TcpServer.create(0) 15 | .start(connection -> 16 | toPublisher(toObservable(connection) 17 | .flatMap(byteBuf -> { 18 | String msg = "Hello " + byteBuf.toString(defaultCharset()); 19 | ByteBuf toWrite = Unpooled.buffer().writeBytes(msg.getBytes()); 20 | return toObservable(connection.write(Publishers.just(toWrite))); 21 | }))); 22 | } 23 | } -------------------------------------------------------------------------------- /ripc-protocol-tcp/src/main/java/io/ripc/protocol/tcp/TcpServer.java: -------------------------------------------------------------------------------- 1 | package io.ripc.protocol.tcp; 2 | 3 | import java.util.concurrent.atomic.AtomicBoolean; 4 | 5 | public abstract class TcpServer { 6 | 7 | protected final TcpHandler thisHandler; 8 | protected final AtomicBoolean started; 9 | 10 | protected TcpServer() { 11 | thisHandler = null; 12 | this.started = new AtomicBoolean(); 13 | } 14 | 15 | public final TcpServer start(TcpHandler handler) { 16 | if (!started.compareAndSet(false, true)) { 17 | throw new IllegalStateException("Server already started"); 18 | } 19 | 20 | doStart(handler); 21 | return this; 22 | } 23 | 24 | public final void startAndAwait(TcpHandler handler) { 25 | start(handler); 26 | awaitShutdown(); 27 | } 28 | 29 | public final boolean shutdown() { 30 | return !started.compareAndSet(true, false) || doShutdown(); 31 | } 32 | 33 | public abstract void awaitShutdown(); 34 | 35 | public abstract boolean doShutdown(); 36 | 37 | protected abstract TcpServer doStart(TcpHandler handler); 38 | 39 | public abstract int getPort(); 40 | 41 | } 42 | -------------------------------------------------------------------------------- /ripc-rxjava1/src/main/java/io/ripc/rx/protocol/tcp/RxTcpServer.java: -------------------------------------------------------------------------------- 1 | package io.ripc.rx.protocol.tcp; 2 | 3 | import io.ripc.protocol.tcp.TcpConnection; 4 | import io.ripc.protocol.tcp.TcpHandler; 5 | import io.ripc.protocol.tcp.TcpServer; 6 | import org.reactivestreams.Publisher; 7 | import rx.RxReactiveStreams; 8 | 9 | public final class RxTcpServer { 10 | 11 | private final TcpServer transport; 12 | 13 | private RxTcpServer(final TcpServer transport) { 14 | this.transport = transport; 15 | } 16 | 17 | public RxTcpServer start(final RxTcpHandler handler) { 18 | 19 | transport.start(new TcpHandler() { 20 | @Override 21 | public Publisher handle(TcpConnection connection) { 22 | return RxReactiveStreams.toPublisher(handler.handle(RxConnection.create(connection))); 23 | } 24 | }); 25 | 26 | return this; 27 | } 28 | 29 | public void startAndAwait(RxTcpHandler handler) { 30 | start(handler); 31 | transport.awaitShutdown(); 32 | } 33 | 34 | public final boolean shutdown() { 35 | return transport.shutdown(); 36 | } 37 | 38 | public void awaitShutdown() { 39 | transport.awaitShutdown(); 40 | } 41 | 42 | public int getPort() { 43 | return transport.getPort(); 44 | } 45 | 46 | public static RxTcpServer create(TcpServer transport) { 47 | return new RxTcpServer<>(transport); 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /ripc-test/src/main/java/io/ripc/test/Publishers.java: -------------------------------------------------------------------------------- 1 | package io.ripc.test; 2 | 3 | import io.ripc.test.internal.PublisherFactory; 4 | import org.reactivestreams.Publisher; 5 | import org.reactivestreams.Subscriber; 6 | import org.reactivestreams.Subscription; 7 | 8 | import java.util.Arrays; 9 | import java.util.List; 10 | 11 | /** 12 | * Temporary utility class for creating and transforming {@link Publisher}s. 13 | */ 14 | public class Publishers { 15 | 16 | public static Publisher just(final T... values) { 17 | final List list = Arrays.asList(values); 18 | return PublisherFactory.forEach( 19 | sub -> { 20 | if (sub.context().hasNext()) { 21 | sub.onNext(sub.context().next()); 22 | } else { 23 | sub.onComplete(); 24 | } 25 | }, 26 | sub -> list.iterator() 27 | ); 28 | } 29 | 30 | public static Publisher error(final Throwable t) { 31 | return new Publisher() { 32 | @Override 33 | public void subscribe(final Subscriber s) { 34 | s.onSubscribe(new Subscription() { 35 | @Override 36 | public void request(long n) { 37 | s.onError(t); 38 | } 39 | 40 | @Override 41 | public void cancel() { 42 | } 43 | }); 44 | } 45 | }; 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /ripc-rxjava1/src/main/java/io/ripc/rx/protocol/tcp/RxConnection.java: -------------------------------------------------------------------------------- 1 | package io.ripc.rx.protocol.tcp; 2 | 3 | import io.ripc.protocol.tcp.TcpConnection; 4 | import rx.Observable; 5 | import rx.Subscriber; 6 | import rx.internal.reactivestreams.SubscriberAdapter; 7 | 8 | import static rx.RxReactiveStreams.*; 9 | 10 | /** 11 | * An adapter for {@link io.ripc.protocol.tcp.TcpConnection} representated as an {@link Observable} 12 | * 13 | * @param The type of objects read from this connection. 14 | * @param The type of objects written to this connection. 15 | */ 16 | public class RxConnection extends Observable { 17 | 18 | private final TcpConnection delegate; 19 | 20 | protected RxConnection(final TcpConnection delegate) { 21 | super(new OnSubscribe() { 22 | @Override 23 | public void call(Subscriber subscriber) { 24 | delegate.subscribe(new SubscriberAdapter<>(subscriber)); 25 | } 26 | }); 27 | this.delegate = delegate; 28 | } 29 | 30 | /** 31 | * Writes the passed stream of {@code data} and returns the result as an {@link Observable}. All items emitted by 32 | * this stream are flushed on completion of the stream. 33 | * 34 | * @param data Data stream to write. 35 | * 36 | * @return Result of write. 37 | */ 38 | public Observable write(Observable data) { 39 | return toObservable(delegate.write(toPublisher(data))); 40 | } 41 | 42 | public static RxConnection create(TcpConnection delegate) { 43 | return new RxConnection<>(delegate); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /ripc-reactor/src/main/java/io/ripc/reactor/protocol/tcp/ReactorTcpServer.java: -------------------------------------------------------------------------------- 1 | package io.ripc.reactor.protocol.tcp; 2 | 3 | import io.ripc.protocol.tcp.TcpConnection; 4 | import io.ripc.protocol.tcp.TcpHandler; 5 | import io.ripc.protocol.tcp.TcpServer; 6 | import org.reactivestreams.Publisher; 7 | import reactor.Environment; 8 | import reactor.fn.Function; 9 | 10 | /** 11 | * Created by jbrisbin on 5/28/15. 12 | */ 13 | public class ReactorTcpServer { 14 | 15 | static { 16 | Environment.initializeIfEmpty().assignErrorJournal(); 17 | } 18 | 19 | private final TcpServer transport; 20 | 21 | ReactorTcpServer(TcpServer transport) { 22 | this.transport = transport; 23 | } 24 | 25 | public ReactorTcpServer start(final ReactorTcpHandler handler) { 26 | 27 | transport.start(new TcpHandler() { 28 | @Override 29 | public Publisher handle(TcpConnection connection) { 30 | return handler.apply(new ReactorTcpConnection<>(connection)); 31 | } 32 | }); 33 | return this; 34 | } 35 | 36 | public ReactorTcpServer startAndAwait(final ReactorTcpHandler handler) { 37 | 38 | transport.startAndAwait(new TcpHandler() { 39 | @Override 40 | public Publisher handle(TcpConnection connection) { 41 | return handler.apply(new ReactorTcpConnection<>(connection)); 42 | } 43 | }); 44 | return this; 45 | } 46 | 47 | public boolean shutdown() { 48 | boolean b = transport.shutdown(); 49 | transport.awaitShutdown(); 50 | return b; 51 | } 52 | 53 | public int getPort() { 54 | return transport.getPort(); 55 | } 56 | 57 | public static ReactorTcpServer create(TcpServer transport) { 58 | return new ReactorTcpServer<>(transport); 59 | } 60 | 61 | } 62 | -------------------------------------------------------------------------------- /ripc-test/src/main/java/io/ripc/test/SocketTestUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2002-2015 the original author or authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.ripc.test; 18 | 19 | import java.io.DataOutputStream; 20 | import java.io.IOException; 21 | import java.io.InputStreamReader; 22 | import java.net.Socket; 23 | 24 | public class SocketTestUtils { 25 | 26 | public static String read(String host, int port) { 27 | return read(host, port, null); 28 | } 29 | 30 | public static String read(String host, int port, String dataToSend) { 31 | try { 32 | Socket socket = new Socket(host, port); 33 | InputStreamReader reader = new InputStreamReader(socket.getInputStream()); 34 | if (dataToSend != null) { 35 | DataOutputStream outputStream = new DataOutputStream(socket.getOutputStream()); 36 | outputStream.writeBytes(dataToSend); 37 | } 38 | StringBuilder content = new StringBuilder(); 39 | int c = reader.read(); 40 | while (c != -1) { 41 | content.append((char)c); 42 | c = reader.read(); 43 | } 44 | reader.close(); 45 | return content.toString(); 46 | } 47 | catch (IOException e) { 48 | throw new IllegalStateException(e); 49 | } 50 | } 51 | 52 | } 53 | -------------------------------------------------------------------------------- /ripc-reactor-examples/src/main/java/io/ripc/reactor/protocol/tcp/ReactorTcpServerSample.java: -------------------------------------------------------------------------------- 1 | package io.ripc.reactor.protocol.tcp; 2 | 3 | import io.netty.buffer.ByteBuf; 4 | import io.netty.buffer.Unpooled; 5 | import io.ripc.protocol.tcp.TcpServer; 6 | import io.ripc.transport.netty4.tcp.Netty4TcpServer; 7 | import reactor.rx.Promise; 8 | import reactor.rx.Promises; 9 | import reactor.rx.Streams; 10 | 11 | import java.nio.charset.Charset; 12 | 13 | /** 14 | * Created by jbrisbin on 5/28/15. 15 | */ 16 | public class ReactorTcpServerSample { 17 | 18 | public static void main(String... args) throws InterruptedException { 19 | TcpServer transport = Netty4TcpServer.create(0); 20 | // echo(transport); 21 | echoWithQuitCommand(transport); 22 | } 23 | 24 | /** 25 | * Keep echoing until the client goes away. 26 | */ 27 | private static void echo(TcpServer transport) { 28 | ReactorTcpServer.create(transport) 29 | .startAndAwait(connection -> { 30 | connection.flatMap(inByteBuf -> { 31 | String text = "Hello " + inByteBuf.toString(Charset.defaultCharset()); 32 | ByteBuf outByteBuf = Unpooled.buffer().writeBytes(text.getBytes()); 33 | return connection.writeWith(Streams.just(outByteBuf)); 34 | }).consume(); 35 | return Streams.never(); 36 | }); 37 | } 38 | 39 | /** 40 | * Keep echoing until the client sends "quit". 41 | */ 42 | private static void echoWithQuitCommand(TcpServer transport) { 43 | ReactorTcpServer.create(transport) 44 | .start(connection -> connection 45 | .map(byteBuf -> byteBuf.toString(Charset.defaultCharset())) 46 | .takeWhile(input -> !"quit".equalsIgnoreCase(input.trim())) 47 | .filter(input -> !"quit".equalsIgnoreCase(input.trim())) 48 | .map(input -> "Hello " + input) 49 | .flatMap(text -> connection.writeWith( 50 | Streams.just(Unpooled.buffer().writeBytes(text.getBytes())) 51 | ) 52 | ) 53 | .after() 54 | ); 55 | } 56 | 57 | } -------------------------------------------------------------------------------- /ripc-rxjava1/src/test/java/io/ripc/rx/RxTcpServerTests.java: -------------------------------------------------------------------------------- 1 | package io.ripc.rx;/* 2 | * Copyright 2002-2015 the original author or authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | import java.io.IOException; 18 | 19 | import io.netty.buffer.ByteBuf; 20 | import io.netty.buffer.Unpooled; 21 | import io.ripc.rx.protocol.tcp.RxTcpServer; 22 | import io.ripc.test.SocketTestUtils; 23 | import io.ripc.transport.netty4.tcp.Netty4TcpServer; 24 | import org.junit.After; 25 | import static org.junit.Assert.assertEquals; 26 | import org.junit.Before; 27 | import org.junit.Test; 28 | import rx.Observable; 29 | 30 | public class RxTcpServerTests { 31 | 32 | private RxTcpServer rxServer; 33 | 34 | @Before 35 | public void setup() { 36 | rxServer = RxTcpServer.create(Netty4TcpServer.create(0)); 37 | } 38 | 39 | @After 40 | public void tearDown() { 41 | rxServer.shutdown(); 42 | } 43 | 44 | @Test 45 | public void writeSingleValue() throws IOException { 46 | rxServer.start(connection -> connection.write(Observable.just(Unpooled.buffer().writeBytes("test".getBytes())))); 47 | assertEquals("test", SocketTestUtils.read("localhost", rxServer.getPort())); 48 | } 49 | 50 | @Test 51 | public void writeMultipleValues() throws IOException { 52 | Observable chunk1 = Observable.just(Unpooled.buffer().writeBytes("This is".getBytes())); 53 | Observable chunk2 = Observable.just(Unpooled.buffer().writeBytes(" a test!".getBytes())); 54 | rxServer.start(connection -> connection.write(Observable.merge(chunk1, chunk2))); 55 | assertEquals("This is a test!", SocketTestUtils.read("localhost", rxServer.getPort())); 56 | } 57 | 58 | } 59 | -------------------------------------------------------------------------------- /ripc-transport-netty4/src/test/java/io/ripc/transport/netty4/TcpServerTests.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2002-2015 the original author or authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.ripc.transport.netty4; 18 | 19 | import java.io.IOException; 20 | 21 | import io.netty.buffer.ByteBuf; 22 | import io.netty.buffer.Unpooled; 23 | import io.ripc.test.Publishers; 24 | import io.ripc.protocol.tcp.TcpServer; 25 | import io.ripc.test.SocketTestUtils; 26 | import io.ripc.transport.netty4.tcp.Netty4TcpServer; 27 | import org.junit.After; 28 | import static org.junit.Assert.assertEquals; 29 | import org.junit.Before; 30 | import org.junit.Test; 31 | 32 | public class TcpServerTests { 33 | 34 | private TcpServer server; 35 | 36 | @Before 37 | public void setup() { 38 | server = Netty4TcpServer.create(0); 39 | } 40 | 41 | @After 42 | public void tearDown() { 43 | server.shutdown(); 44 | } 45 | 46 | @Test 47 | public void writeSingleValue() throws IOException { 48 | server.start(connection -> connection.write(Publishers.just(Unpooled.buffer().writeBytes("test".getBytes())))); 49 | assertEquals("test", SocketTestUtils.read("localhost", server.getPort())); 50 | } 51 | 52 | @Test 53 | public void writeMultipleValues() throws IOException { 54 | server.start(connection -> { 55 | ByteBuf chunk1 = Unpooled.buffer().writeBytes("This is".getBytes()); 56 | ByteBuf chunk2 = Unpooled.buffer().writeBytes(" a test!".getBytes()); 57 | return connection.write(Publishers.just(chunk1, chunk2)); 58 | }); 59 | assertEquals("This is a test!", SocketTestUtils.read("localhost", server.getPort())); 60 | } 61 | 62 | } 63 | -------------------------------------------------------------------------------- /ripc-transport-netty4/src/main/java/io/ripc/transport/netty4/tcp/TcpConnectionImpl.java: -------------------------------------------------------------------------------- 1 | package io.ripc.transport.netty4.tcp; 2 | 3 | import io.netty.channel.Channel; 4 | import io.netty.channel.ChannelFuture; 5 | import io.netty.channel.ChannelFutureListener; 6 | import io.ripc.protocol.tcp.TcpConnection; 7 | import io.ripc.transport.netty4.tcp.ChannelToConnectionBridge.ConnectionInputSubscriberEvent; 8 | import org.reactivestreams.Publisher; 9 | import org.reactivestreams.Subscriber; 10 | import org.reactivestreams.Subscription; 11 | 12 | public class TcpConnectionImpl implements TcpConnection { 13 | 14 | private final Channel nettyChannel; 15 | 16 | public TcpConnectionImpl(Channel nettyChannel) { 17 | this.nettyChannel = nettyChannel; 18 | } 19 | 20 | @Override 21 | public Publisher write(final Publisher data) { 22 | return new Publisher() { 23 | @Override 24 | public void subscribe(Subscriber s) { 25 | nettyChannel.write(data).addListener(new FutureToSubscriberBridge(s)); 26 | } 27 | }; 28 | } 29 | 30 | @Override 31 | public void subscribe(Subscriber s) { 32 | nettyChannel.pipeline().fireUserEventTriggered(new ConnectionInputSubscriberEvent<>(s)); 33 | } 34 | 35 | private static final Subscription IGNORE_SUBSCRIPTION = new Subscription() { 36 | @Override 37 | public void request(long n) { 38 | //IGNORE 39 | } 40 | 41 | @Override 42 | public void cancel() { 43 | //IGNORE 44 | } 45 | }; 46 | 47 | private static class FutureToSubscriberBridge implements ChannelFutureListener { 48 | 49 | private final Subscriber subscriber; 50 | 51 | public FutureToSubscriberBridge(Subscriber subscriber) { 52 | this.subscriber = subscriber; 53 | subscriber.onSubscribe(IGNORE_SUBSCRIPTION); 54 | } 55 | 56 | @Override 57 | public void operationComplete(ChannelFuture future) throws Exception { 58 | if (future.isSuccess()) { 59 | subscriber.onComplete(); 60 | } else { 61 | subscriber.onError(future.cause()); 62 | } 63 | } 64 | } 65 | 66 | } 67 | -------------------------------------------------------------------------------- /ripc-reactor/src/test/java/io/ripc/reactor/ReactorTcpServerTests.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2002-2015 the original author or authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.ripc.reactor; 18 | 19 | import java.io.IOException; 20 | 21 | import io.netty.buffer.ByteBuf; 22 | import io.netty.buffer.Unpooled; 23 | import io.ripc.reactor.protocol.tcp.ReactorTcpServer; 24 | import io.ripc.test.SocketTestUtils; 25 | import io.ripc.transport.netty4.tcp.Netty4TcpServer; 26 | import org.junit.After; 27 | import static org.junit.Assert.assertEquals; 28 | import org.junit.Before; 29 | import org.junit.Test; 30 | import reactor.rx.Promise; 31 | import reactor.rx.Promises; 32 | import reactor.rx.Streams; 33 | 34 | public class ReactorTcpServerTests { 35 | 36 | private ReactorTcpServer reactorServer; 37 | 38 | @Before 39 | public void setup() { 40 | reactorServer = ReactorTcpServer.create(Netty4TcpServer.create(0)); 41 | } 42 | 43 | @After 44 | public void tearDown() { 45 | reactorServer.shutdown(); 46 | } 47 | 48 | @Test 49 | public void writeSingleValue() throws IOException { 50 | reactorServer.start(connection -> connection.writeWith(Streams.just(Unpooled.buffer().writeBytes("test".getBytes())))); 51 | assertEquals("test", SocketTestUtils.read("localhost", reactorServer.getPort())); 52 | } 53 | 54 | @Test 55 | public void writeMultipleValues() throws IOException { 56 | Promise chunk1 = Promises.success(Unpooled.buffer().writeBytes("This is".getBytes())); 57 | Promise chunk2 = Promises.success(Unpooled.buffer().writeBytes(" a test!".getBytes())); 58 | reactorServer.start(connection -> connection.writeWith(Streams.concat(chunk1, chunk2))); 59 | assertEquals("This is a test!", SocketTestUtils.read("localhost", reactorServer.getPort())); 60 | } 61 | 62 | } 63 | -------------------------------------------------------------------------------- /gradlew.bat: -------------------------------------------------------------------------------- 1 | @if "%DEBUG%" == "" @echo off 2 | @rem ########################################################################## 3 | @rem 4 | @rem Gradle startup script for Windows 5 | @rem 6 | @rem ########################################################################## 7 | 8 | @rem Set local scope for the variables with windows NT shell 9 | if "%OS%"=="Windows_NT" setlocal 10 | 11 | @rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. 12 | set DEFAULT_JVM_OPTS= 13 | 14 | set DIRNAME=%~dp0 15 | if "%DIRNAME%" == "" set DIRNAME=. 16 | set APP_BASE_NAME=%~n0 17 | set APP_HOME=%DIRNAME% 18 | 19 | @rem Find java.exe 20 | if defined JAVA_HOME goto findJavaFromJavaHome 21 | 22 | set JAVA_EXE=java.exe 23 | %JAVA_EXE% -version >NUL 2>&1 24 | if "%ERRORLEVEL%" == "0" goto init 25 | 26 | echo. 27 | echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. 28 | echo. 29 | echo Please set the JAVA_HOME variable in your environment to match the 30 | echo location of your Java installation. 31 | 32 | goto fail 33 | 34 | :findJavaFromJavaHome 35 | set JAVA_HOME=%JAVA_HOME:"=% 36 | set JAVA_EXE=%JAVA_HOME%/bin/java.exe 37 | 38 | if exist "%JAVA_EXE%" goto init 39 | 40 | echo. 41 | echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME% 42 | echo. 43 | echo Please set the JAVA_HOME variable in your environment to match the 44 | echo location of your Java installation. 45 | 46 | goto fail 47 | 48 | :init 49 | @rem Get command-line arguments, handling Windowz variants 50 | 51 | if not "%OS%" == "Windows_NT" goto win9xME_args 52 | if "%@eval[2+2]" == "4" goto 4NT_args 53 | 54 | :win9xME_args 55 | @rem Slurp the command line arguments. 56 | set CMD_LINE_ARGS= 57 | set _SKIP=2 58 | 59 | :win9xME_args_slurp 60 | if "x%~1" == "x" goto execute 61 | 62 | set CMD_LINE_ARGS=%* 63 | goto execute 64 | 65 | :4NT_args 66 | @rem Get arguments from the 4NT Shell from JP Software 67 | set CMD_LINE_ARGS=%$ 68 | 69 | :execute 70 | @rem Setup the command line 71 | 72 | set CLASSPATH=%APP_HOME%\gradle\wrapper\gradle-wrapper.jar 73 | 74 | @rem Execute Gradle 75 | "%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -classpath "%CLASSPATH%" org.gradle.wrapper.GradleWrapperMain %CMD_LINE_ARGS% 76 | 77 | :end 78 | @rem End local scope for the variables with windows NT shell 79 | if "%ERRORLEVEL%"=="0" goto mainEnd 80 | 81 | :fail 82 | rem Set variable GRADLE_EXIT_CONSOLE if you need the _script_ return code instead of 83 | rem the _cmd.exe /c_ return code! 84 | if not "" == "%GRADLE_EXIT_CONSOLE%" exit 1 85 | exit /b 1 86 | 87 | :mainEnd 88 | if "%OS%"=="Windows_NT" endlocal 89 | 90 | :omega 91 | -------------------------------------------------------------------------------- /ripc-core/src/main/java/io/ripc/io/Buffer.java: -------------------------------------------------------------------------------- 1 | package io.ripc.io; 2 | 3 | import java.nio.ByteBuffer; 4 | import java.nio.charset.CharsetDecoder; 5 | import java.util.List; 6 | 7 | /** 8 | * Common abstraction to provide additional functionality beyond a traditional {@link java.nio.ByteBuffer} while not 9 | * restricting the dedicated functionality provided by concrete implementations from various transport libraries that 10 | * might offer features like zero-copy. 11 | *

12 | * A {@code Buffer} can be anything. It is not limited to byte buffers. A {@code Buffer} could represent realized 13 | * objects descended from raw data. 14 | *

15 | */ 16 | public interface Buffer extends Cloneable, 17 | AutoCloseable, 18 | Comparable>, 19 | Iterable> { 20 | 21 | int position(); 22 | 23 | Buffer position(int pos); 24 | 25 | int limit(); 26 | 27 | Buffer limit(int limit); 28 | 29 | int capacity(); 30 | 31 | Buffer capacity(int capacity); 32 | 33 | int remaining(); 34 | 35 | Buffer skip(int len); 36 | 37 | Buffer clear(); 38 | 39 | Buffer compact(); 40 | 41 | Buffer flip(); 42 | 43 | Buffer rewind(); 44 | 45 | Buffer rewind(int len); 46 | 47 | Buffer clone(); 48 | 49 | Buffer copy(); 50 | 51 | Buffer slice(int start, int len); 52 | 53 | Iterable> split(byte delimiter); 54 | 55 | Iterable> split(byte delimiter, boolean stripDelimiter); 56 | 57 | Iterable> split(byte delimiter, boolean stripDelimiter, List> preallocatedList); 58 | 59 | Iterable> split(Buffer delimiter); 60 | 61 | Iterable> split(Buffer delimiter, boolean stripDelimiter); 62 | 63 | Iterable> split(Buffer delimiter, boolean stripDelimiter, List> preallocatedList); 64 | 65 | Buffer prepend(B data); 66 | 67 | Buffer prepend(Buffer buffer); 68 | 69 | Buffer prepend(ByteBuffer buffer); 70 | 71 | Buffer prepend(CharSequence chars); 72 | 73 | Buffer prepend(byte[] bytes); 74 | 75 | Buffer prepend(byte b); 76 | 77 | Buffer prepend(char c); 78 | 79 | Buffer prepend(short s); 80 | 81 | Buffer prepend(int i); 82 | 83 | Buffer prepend(long l); 84 | 85 | Buffer append(B data); 86 | 87 | Buffer append(Buffer buffer); 88 | 89 | Buffer append(ByteBuffer buffer); 90 | 91 | Buffer append(CharSequence chars); 92 | 93 | Buffer append(byte[] bytes); 94 | 95 | Buffer append(byte b); 96 | 97 | Buffer append(char c); 98 | 99 | Buffer append(short s); 100 | 101 | Buffer append(int i); 102 | 103 | Buffer append(long l); 104 | 105 | byte readByte(); 106 | 107 | void readBytes(byte[] bytes); 108 | 109 | short readShort(); 110 | 111 | int readInt(); 112 | 113 | float readFloat(); 114 | 115 | double readDouble(); 116 | 117 | long readLong(); 118 | 119 | char readChar(); 120 | 121 | void readChars(char[] chars); 122 | 123 | String readString(); 124 | 125 | String readString(CharsetDecoder decoder); 126 | 127 | B get(); 128 | 129 | } 130 | -------------------------------------------------------------------------------- /ripc-transport-netty4/src/main/java/io/ripc/transport/netty4/tcp/Netty4TcpServer.java: -------------------------------------------------------------------------------- 1 | package io.ripc.transport.netty4.tcp; 2 | 3 | import io.netty.bootstrap.ServerBootstrap; 4 | import io.netty.channel.Channel; 5 | import io.netty.channel.ChannelFuture; 6 | import io.netty.channel.ChannelInitializer; 7 | import io.netty.channel.nio.NioEventLoopGroup; 8 | import io.netty.channel.socket.nio.NioServerSocketChannel; 9 | import io.ripc.protocol.tcp.TcpHandler; 10 | import io.ripc.protocol.tcp.TcpServer; 11 | import org.slf4j.Logger; 12 | import org.slf4j.LoggerFactory; 13 | 14 | import java.net.InetSocketAddress; 15 | import java.net.SocketAddress; 16 | 17 | public class Netty4TcpServer extends TcpServer { 18 | 19 | private static final Logger logger = LoggerFactory.getLogger(Netty4TcpServer.class); 20 | 21 | private int port; 22 | private final ChannelInitializer initializer; 23 | private ServerBootstrap bootstrap; 24 | private ChannelFuture bindFuture; 25 | 26 | protected Netty4TcpServer(int port) { 27 | this(port, null); 28 | } 29 | 30 | protected Netty4TcpServer(int port, ChannelInitializer initializer) { 31 | this.port = port; 32 | this.initializer = initializer; 33 | bootstrap = new ServerBootstrap() 34 | .group(new NioEventLoopGroup()) 35 | .channel(NioServerSocketChannel.class); 36 | } 37 | 38 | @Override 39 | protected Netty4TcpServer doStart(final TcpHandler handler) { 40 | bootstrap.childHandler(new ChannelInitializer() { 41 | @Override 42 | protected void initChannel(Channel ch) throws Exception { 43 | if (initializer != null) { 44 | ch.pipeline().addLast(initializer); 45 | } 46 | ch.config().setAutoRead(false); 47 | ch.pipeline().addLast("server_handler", new ChannelToConnectionBridge<>(handler)); 48 | } 49 | }); 50 | 51 | try { 52 | bindFuture = bootstrap.bind(port).sync(); 53 | if (!bindFuture.isSuccess()) { 54 | throw new RuntimeException(bindFuture.cause()); 55 | } 56 | SocketAddress localAddress = bindFuture.channel().localAddress(); 57 | if (localAddress instanceof InetSocketAddress) { 58 | port = ((InetSocketAddress) localAddress).getPort(); 59 | logger.info("Started server at port: " + port); 60 | } 61 | 62 | } catch (InterruptedException e) { 63 | logger.error("Error waiting for binding server port: " + port, e); 64 | } 65 | 66 | return this; 67 | } 68 | 69 | @Override 70 | public void awaitShutdown() { 71 | try { 72 | bindFuture.channel().closeFuture().await(); 73 | } catch (InterruptedException e) { 74 | Thread.interrupted(); // Reset the interrupted status 75 | logger.error("Interrupted while waiting for the server socket to close.", e); 76 | } 77 | } 78 | 79 | @Override 80 | public boolean doShutdown() { 81 | try { 82 | bindFuture.channel().close().sync(); 83 | return true; 84 | } catch (InterruptedException e) { 85 | logger.error("Failed to shutdown the server.", e); 86 | return false; 87 | } 88 | } 89 | 90 | @Override 91 | public int getPort() { 92 | return port; 93 | } 94 | 95 | public static TcpServer create(int port) { 96 | return new Netty4TcpServer<>(port); 97 | } 98 | 99 | public static TcpServer create(int port, ChannelInitializer initializer) { 100 | return new Netty4TcpServer(port, initializer); 101 | } 102 | 103 | } 104 | -------------------------------------------------------------------------------- /ripc-test/src/main/java/io/ripc/test/internal/SubscriberWithContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 Pivotal Software Inc, All Rights Reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package io.ripc.test.internal; 17 | 18 | import org.reactivestreams.Subscriber; 19 | import org.reactivestreams.Subscription; 20 | 21 | import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; 22 | 23 | /** 24 | * Original {@see reactor.core.reactivestreams.PublisherFactory} from {@see http://projectreactor.io}. 25 | *

26 | * A {@link Subscriber} with a typed stateful context. Some error isolation is also provided 27 | * (onSubscribe, onNext and onComplete error is forwarded to onError). 28 | * 29 | * @author Stephane Maldini 30 | */ 31 | public class SubscriberWithContext implements Subscriber { 32 | 33 | private volatile int terminated = 0; 34 | protected static final AtomicIntegerFieldUpdater TERMINAL_UPDATER = 35 | AtomicIntegerFieldUpdater 36 | .newUpdater(SubscriberWithContext.class, "terminated"); 37 | 38 | 39 | protected final C context; 40 | protected final Subscriber subscriber; 41 | 42 | /** 43 | * Attach a given arbitrary context (stateful information) to a {@link Subscriber}, all Subscriber methods 44 | * will delegate properly. 45 | * 46 | * @param subscriber the delegate subscriber to invoke on signal 47 | * @param context the contextual state of any type to bind for later use 48 | * @param Type of data sequence 49 | * @param Type of attached stateful context 50 | * @return a new Susbscriber with context information 51 | */ 52 | public static SubscriberWithContext create(Subscriber subscriber, C context) { 53 | return new SubscriberWithContext<>(context, subscriber); 54 | } 55 | 56 | protected SubscriberWithContext(C context, Subscriber subscriber) { 57 | this.context = context; 58 | this.subscriber = subscriber; 59 | } 60 | 61 | /** 62 | * The stateful context C 63 | * 64 | * @return the bound context 65 | */ 66 | public C context() { 67 | return context; 68 | } 69 | 70 | @Override 71 | public void onSubscribe(Subscription s) { 72 | try { 73 | subscriber.onSubscribe(s); 74 | } catch (Throwable throwable) { 75 | subscriber.onError(throwable); 76 | } 77 | } 78 | 79 | @Override 80 | public void onNext(T t) { 81 | try { 82 | subscriber.onNext(t); 83 | } catch (Throwable throwable) { 84 | subscriber.onError(throwable); 85 | } 86 | } 87 | 88 | @Override 89 | public void onError(Throwable t) { 90 | if (TERMINAL_UPDATER.compareAndSet(this, 0, 1)) { 91 | subscriber.onError(t); 92 | } 93 | } 94 | 95 | @Override 96 | public void onComplete() { 97 | try { 98 | if (TERMINAL_UPDATER.compareAndSet(this, 0, 1)) { 99 | subscriber.onComplete(); 100 | } 101 | } catch (Throwable throwable) { 102 | subscriber.onError(throwable); 103 | } 104 | } 105 | 106 | public boolean isCancelled() { 107 | return terminated == 1; 108 | } 109 | } 110 | -------------------------------------------------------------------------------- /gradlew: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | ############################################################################## 4 | ## 5 | ## Gradle start up script for UN*X 6 | ## 7 | ############################################################################## 8 | 9 | # Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. 10 | DEFAULT_JVM_OPTS="" 11 | 12 | APP_NAME="Gradle" 13 | APP_BASE_NAME=`basename "$0"` 14 | 15 | # Use the maximum available, or set MAX_FD != -1 to use that value. 16 | MAX_FD="maximum" 17 | 18 | warn ( ) { 19 | echo "$*" 20 | } 21 | 22 | die ( ) { 23 | echo 24 | echo "$*" 25 | echo 26 | exit 1 27 | } 28 | 29 | # OS specific support (must be 'true' or 'false'). 30 | cygwin=false 31 | msys=false 32 | darwin=false 33 | case "`uname`" in 34 | CYGWIN* ) 35 | cygwin=true 36 | ;; 37 | Darwin* ) 38 | darwin=true 39 | ;; 40 | MINGW* ) 41 | msys=true 42 | ;; 43 | esac 44 | 45 | # For Cygwin, ensure paths are in UNIX format before anything is touched. 46 | if $cygwin ; then 47 | [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --unix "$JAVA_HOME"` 48 | fi 49 | 50 | # Attempt to set APP_HOME 51 | # Resolve links: $0 may be a link 52 | PRG="$0" 53 | # Need this for relative symlinks. 54 | while [ -h "$PRG" ] ; do 55 | ls=`ls -ld "$PRG"` 56 | link=`expr "$ls" : '.*-> \(.*\)$'` 57 | if expr "$link" : '/.*' > /dev/null; then 58 | PRG="$link" 59 | else 60 | PRG=`dirname "$PRG"`"/$link" 61 | fi 62 | done 63 | SAVED="`pwd`" 64 | cd "`dirname \"$PRG\"`/" >&- 65 | APP_HOME="`pwd -P`" 66 | cd "$SAVED" >&- 67 | 68 | CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar 69 | 70 | # Determine the Java command to use to start the JVM. 71 | if [ -n "$JAVA_HOME" ] ; then 72 | if [ -x "$JAVA_HOME/jre/sh/java" ] ; then 73 | # IBM's JDK on AIX uses strange locations for the executables 74 | JAVACMD="$JAVA_HOME/jre/sh/java" 75 | else 76 | JAVACMD="$JAVA_HOME/bin/java" 77 | fi 78 | if [ ! -x "$JAVACMD" ] ; then 79 | die "ERROR: JAVA_HOME is set to an invalid directory: $JAVA_HOME 80 | 81 | Please set the JAVA_HOME variable in your environment to match the 82 | location of your Java installation." 83 | fi 84 | else 85 | JAVACMD="java" 86 | which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. 87 | 88 | Please set the JAVA_HOME variable in your environment to match the 89 | location of your Java installation." 90 | fi 91 | 92 | # Increase the maximum file descriptors if we can. 93 | if [ "$cygwin" = "false" -a "$darwin" = "false" ] ; then 94 | MAX_FD_LIMIT=`ulimit -H -n` 95 | if [ $? -eq 0 ] ; then 96 | if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then 97 | MAX_FD="$MAX_FD_LIMIT" 98 | fi 99 | ulimit -n $MAX_FD 100 | if [ $? -ne 0 ] ; then 101 | warn "Could not set maximum file descriptor limit: $MAX_FD" 102 | fi 103 | else 104 | warn "Could not query maximum file descriptor limit: $MAX_FD_LIMIT" 105 | fi 106 | fi 107 | 108 | # For Darwin, add options to specify how the application appears in the dock 109 | if $darwin; then 110 | GRADLE_OPTS="$GRADLE_OPTS \"-Xdock:name=$APP_NAME\" \"-Xdock:icon=$APP_HOME/media/gradle.icns\"" 111 | fi 112 | 113 | # For Cygwin, switch paths to Windows format before running java 114 | if $cygwin ; then 115 | APP_HOME=`cygpath --path --mixed "$APP_HOME"` 116 | CLASSPATH=`cygpath --path --mixed "$CLASSPATH"` 117 | 118 | # We build the pattern for arguments to be converted via cygpath 119 | ROOTDIRSRAW=`find -L / -maxdepth 1 -mindepth 1 -type d 2>/dev/null` 120 | SEP="" 121 | for dir in $ROOTDIRSRAW ; do 122 | ROOTDIRS="$ROOTDIRS$SEP$dir" 123 | SEP="|" 124 | done 125 | OURCYGPATTERN="(^($ROOTDIRS))" 126 | # Add a user-defined pattern to the cygpath arguments 127 | if [ "$GRADLE_CYGPATTERN" != "" ] ; then 128 | OURCYGPATTERN="$OURCYGPATTERN|($GRADLE_CYGPATTERN)" 129 | fi 130 | # Now convert the arguments - kludge to limit ourselves to /bin/sh 131 | i=0 132 | for arg in "$@" ; do 133 | CHECK=`echo "$arg"|egrep -c "$OURCYGPATTERN" -` 134 | CHECK2=`echo "$arg"|egrep -c "^-"` ### Determine if an option 135 | 136 | if [ $CHECK -ne 0 ] && [ $CHECK2 -eq 0 ] ; then ### Added a condition 137 | eval `echo args$i`=`cygpath --path --ignore --mixed "$arg"` 138 | else 139 | eval `echo args$i`="\"$arg\"" 140 | fi 141 | i=$((i+1)) 142 | done 143 | case $i in 144 | (0) set -- ;; 145 | (1) set -- "$args0" ;; 146 | (2) set -- "$args0" "$args1" ;; 147 | (3) set -- "$args0" "$args1" "$args2" ;; 148 | (4) set -- "$args0" "$args1" "$args2" "$args3" ;; 149 | (5) set -- "$args0" "$args1" "$args2" "$args3" "$args4" ;; 150 | (6) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" ;; 151 | (7) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" ;; 152 | (8) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" ;; 153 | (9) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" "$args8" ;; 154 | esac 155 | fi 156 | 157 | # Split up the JVM_OPTS And GRADLE_OPTS values into an array, following the shell quoting and substitution rules 158 | function splitJvmOpts() { 159 | JVM_OPTS=("$@") 160 | } 161 | eval splitJvmOpts $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS 162 | JVM_OPTS[${#JVM_OPTS[*]}]="-Dorg.gradle.appname=$APP_BASE_NAME" 163 | 164 | exec "$JAVACMD" "${JVM_OPTS[@]}" -classpath "$CLASSPATH" org.gradle.wrapper.GradleWrapperMain "$@" 165 | -------------------------------------------------------------------------------- /ripc-reactor-examples/src/main/java/io/ripc/reactor/protocol/tcp/CodecSample.java: -------------------------------------------------------------------------------- 1 | package io.ripc.reactor.protocol.tcp; 2 | 3 | import com.fasterxml.jackson.databind.ObjectMapper; 4 | import io.netty.buffer.ByteBuf; 5 | import io.netty.channel.Channel; 6 | import io.netty.channel.ChannelConfig; 7 | import io.netty.channel.ChannelDuplexHandler; 8 | import io.netty.channel.ChannelHandlerContext; 9 | import io.netty.channel.ChannelInitializer; 10 | import io.netty.channel.ChannelOption; 11 | import io.netty.channel.ChannelPromise; 12 | import io.netty.channel.FixedRecvByteBufAllocator; 13 | import io.netty.handler.codec.LineBasedFrameDecoder; 14 | import io.netty.handler.codec.json.JsonObjectDecoder; 15 | import io.netty.handler.codec.string.StringDecoder; 16 | import io.netty.handler.codec.string.StringEncoder; 17 | import io.netty.util.CharsetUtil; 18 | import io.ripc.protocol.tcp.TcpServer; 19 | import io.ripc.transport.netty4.tcp.Netty4TcpServer; 20 | import org.slf4j.Logger; 21 | import org.slf4j.LoggerFactory; 22 | import reactor.rx.Streams; 23 | 24 | import java.nio.charset.Charset; 25 | 26 | public class CodecSample { 27 | 28 | private static final Logger LOG = LoggerFactory.getLogger(CodecSample.class); 29 | 30 | 31 | public static void main(String... args) throws InterruptedException { 32 | //runLineBasedFrameDecoder(); 33 | echoJsonStreamDecoding(); 34 | } 35 | 36 | private static void runLineBasedFrameDecoder() { 37 | 38 | TcpServer transport = Netty4TcpServer.create( 39 | 0, 40 | new ChannelInitializer() { 41 | @Override 42 | protected void initChannel(Channel channel) throws Exception { 43 | int bufferSize = 1; 44 | ChannelConfig config = channel.config(); 45 | config.setOption(ChannelOption.SO_RCVBUF, bufferSize); 46 | config.setOption(ChannelOption.RCVBUF_ALLOCATOR, new FixedRecvByteBufAllocator(bufferSize)); 47 | channel.pipeline().addFirst( 48 | new LineBasedFrameDecoder(256), 49 | new StringDecoder(CharsetUtil.UTF_8), 50 | new StringEncoder(CharsetUtil.UTF_8)); 51 | } 52 | }); 53 | 54 | ReactorTcpServer.create(transport).start(connection -> { 55 | connection.log("input") 56 | .observeComplete(v -> LOG.info("Connection input complete")) 57 | .capacity(1) 58 | .consume(line -> { 59 | String response = "Hello " + line + "\n"; 60 | Streams.wrap(connection.writeWith(Streams.just(response))).consume(); 61 | }); 62 | return Streams.never(); 63 | }); 64 | } 65 | 66 | private static void echoJsonStreamDecoding() { 67 | 68 | TcpServer transport = Netty4TcpServer.create( 69 | 0, 70 | new ChannelInitializer() { 71 | @Override 72 | protected void initChannel(Channel channel) throws Exception { 73 | channel.pipeline().addFirst( 74 | new JsonObjectDecoder(), 75 | new JacksonJsonCodec()); 76 | } 77 | }); 78 | 79 | ReactorTcpServer.create(transport) 80 | .start(connection -> { 81 | connection.log("input") 82 | .observeComplete(v -> LOG.info("Connection input complete")) 83 | .capacity(1) 84 | .consume(person -> { 85 | person = new Person(person.getLastName(), person.getFirstName()); 86 | Streams.wrap(connection.writeWith(Streams.just(person))).consume(); 87 | }); 88 | return Streams.never(); 89 | }); 90 | 91 | } 92 | 93 | private static class JacksonJsonCodec extends ChannelDuplexHandler { 94 | 95 | private final ObjectMapper mapper = new ObjectMapper(); 96 | 97 | @Override 98 | public void channelRead(ChannelHandlerContext context, Object message) throws Exception { 99 | if (message instanceof ByteBuf) { 100 | Charset charset = Charset.defaultCharset(); 101 | message = this.mapper.readValue(((ByteBuf) message).toString(charset), Person.class); 102 | } 103 | super.channelRead(context, message); 104 | } 105 | 106 | @Override 107 | public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { 108 | if (msg instanceof Person) { 109 | byte[] buff = mapper.writeValueAsBytes(msg); 110 | ByteBuf bb = ctx.alloc().buffer(buff.length); 111 | bb.writeBytes(buff); 112 | msg = bb; 113 | } 114 | super.write(ctx, msg, promise); 115 | } 116 | } 117 | 118 | private static class Person { 119 | 120 | private String firstName; 121 | 122 | private String lastName; 123 | 124 | public Person() { 125 | } 126 | 127 | public Person(String firstName, String lastName) { 128 | this.firstName = firstName; 129 | this.lastName = lastName; 130 | } 131 | 132 | public String getFirstName() { 133 | return firstName; 134 | } 135 | 136 | public Person setFirstName(String firstName) { 137 | this.firstName = firstName; 138 | return this; 139 | } 140 | 141 | public String getLastName() { 142 | return lastName; 143 | } 144 | 145 | public Person setLastName(String lastName) { 146 | this.lastName = lastName; 147 | return this; 148 | } 149 | } 150 | 151 | } 152 | -------------------------------------------------------------------------------- /ripc-transport-netty4/src/main/java/io/ripc/transport/netty4/tcp/ChannelToConnectionBridge.java: -------------------------------------------------------------------------------- 1 | package io.ripc.transport.netty4.tcp; 2 | 3 | import io.netty.channel.Channel; 4 | import io.netty.channel.ChannelDuplexHandler; 5 | import io.netty.channel.ChannelFuture; 6 | import io.netty.channel.ChannelFutureListener; 7 | import io.netty.channel.ChannelHandlerContext; 8 | import io.netty.channel.ChannelPromise; 9 | import io.netty.util.ReferenceCountUtil; 10 | import io.ripc.protocol.tcp.TcpHandler; 11 | import org.reactivestreams.Publisher; 12 | import org.reactivestreams.Subscriber; 13 | import org.reactivestreams.Subscription; 14 | import org.slf4j.Logger; 15 | import org.slf4j.LoggerFactory; 16 | 17 | /** 18 | * A bridge between netty's {@link Channel} and {@link io.ripc.protocol.tcp.TcpConnection}. It has the following 19 | * responsibilities: 20 | *

21 | *

    22 | *
  • Create a new {@link io.ripc.protocol.tcp.TcpConnection} instance when the channel is active and forwards it to 23 | * the configured 24 | * {@link TcpHandler}.
  • 25 | *
  • Reads any data from the channel and forwards it to the {@link Subscriber} attached via the event 26 | * {@link ChannelToConnectionBridge.ConnectionInputSubscriberEvent}
  • 27 | *
  • Accepts writes of {@link Publisher} on the channel and translates the items emitted from that publisher to the 28 | * channel.
  • 29 | *
30 | * 31 | * @param The type of objects read from the underneath channel. 32 | * @param The type of objects read written to the underneath channel. 33 | */ 34 | public class ChannelToConnectionBridge extends ChannelDuplexHandler { 35 | 36 | private static final Logger logger = LoggerFactory.getLogger(ChannelToConnectionBridge.class); 37 | 38 | private final TcpHandler handler; 39 | private TcpConnectionImpl conn; 40 | private Subscriber inputSubscriber; /*Populated via event ConnectionInputSubscriberEvent*/ 41 | 42 | public ChannelToConnectionBridge(TcpHandler handler) { 43 | this.handler = handler; 44 | } 45 | 46 | @Override 47 | public void channelActive(final ChannelHandlerContext ctx) throws Exception { 48 | super.channelActive(ctx); 49 | conn = new TcpConnectionImpl<>(ctx.channel()); 50 | handler.handle(conn) 51 | .subscribe(new Subscriber() { 52 | @Override 53 | public void onSubscribe(Subscription s) { 54 | s.request(Long.MAX_VALUE); //no op 55 | } 56 | 57 | @Override 58 | public void onNext(Void aVoid) { 59 | // Void, no op 60 | } 61 | 62 | @Override 63 | public void onError(Throwable t) { 64 | logger.error("Error processing connection. Closing the channel.", t); 65 | ctx.channel().close(); 66 | } 67 | 68 | @Override 69 | public void onComplete() { 70 | ctx.channel().close(); 71 | } 72 | }); 73 | } 74 | 75 | @SuppressWarnings("unchecked") 76 | @Override 77 | public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { 78 | if (null == conn || null == inputSubscriber) { 79 | logger.error("No connection input subscriber available. Disposing data."); 80 | ReferenceCountUtil.release(msg); 81 | return; 82 | } 83 | 84 | try { 85 | inputSubscriber.onNext((R) msg); 86 | } catch (ClassCastException e) { 87 | logger.error("Invalid message type read from the pipeline.", e); 88 | inputSubscriber.onError(e); 89 | } 90 | } 91 | 92 | @Override 93 | public void channelInactive(ChannelHandlerContext ctx) throws Exception { 94 | if (null != conn && inputSubscriber != null) { 95 | inputSubscriber.onComplete(); 96 | } 97 | super.channelInactive(ctx); 98 | } 99 | 100 | @Override 101 | public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { 102 | if (evt instanceof ConnectionInputSubscriberEvent) { 103 | @SuppressWarnings("unchecked") 104 | ConnectionInputSubscriberEvent subscriberEvent = (ConnectionInputSubscriberEvent) evt; 105 | if (null == inputSubscriber) { 106 | inputSubscriber = subscriberEvent.getInputSubscriber(); 107 | subscriberEvent.init(ctx); 108 | } else { 109 | inputSubscriber.onError(new IllegalStateException("Only one connection input subscriber allowed.")); 110 | } 111 | } 112 | super.userEventTriggered(ctx, evt); 113 | } 114 | 115 | @Override 116 | public void write(final ChannelHandlerContext ctx, Object msg, final ChannelPromise promise) throws Exception { 117 | if (msg instanceof Publisher) { 118 | @SuppressWarnings("unchecked") 119 | final Publisher data = (Publisher) msg; 120 | 121 | data.subscribe(new Subscriber() { 122 | 123 | // TODO: Needs to be fixed to wire all futures to the promise of the Publisher write. 124 | private ChannelFuture lastWriteFuture; 125 | 126 | @Override 127 | public void onSubscribe(Subscription s) { 128 | s.request(Long.MAX_VALUE); // TODO: Backpressure 129 | } 130 | 131 | @Override 132 | public void onNext(W w) { 133 | lastWriteFuture = ctx.channel().write(w); 134 | } 135 | 136 | @Override 137 | public void onError(Throwable t) { 138 | onTerminate(); 139 | } 140 | 141 | @Override 142 | public void onComplete() { 143 | onTerminate(); 144 | } 145 | 146 | private void onTerminate() { 147 | ctx.channel().flush(); 148 | lastWriteFuture.addListener(new ChannelFutureListener() { 149 | @Override 150 | public void operationComplete(ChannelFuture future) throws Exception { 151 | if (future.isSuccess()) { 152 | promise.trySuccess(); 153 | } else { 154 | promise.tryFailure(future.cause()); 155 | } 156 | } 157 | }); 158 | } 159 | }); 160 | } else { 161 | super.write(ctx, msg, promise); 162 | } 163 | } 164 | 165 | @Override 166 | public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { 167 | logger.error(cause.getMessage(), cause); 168 | } 169 | 170 | /** 171 | * An event to attach a {@link Subscriber} to the {@link io.ripc.protocol.tcp.TcpConnection} created by {@link 172 | * ChannelToConnectionBridge} 173 | * 174 | * @param 175 | */ 176 | public static final class ConnectionInputSubscriberEvent { 177 | 178 | private final Subscriber inputSubscriber; 179 | 180 | public ConnectionInputSubscriberEvent(Subscriber inputSubscriber) { 181 | if (null == inputSubscriber) { 182 | throw new IllegalArgumentException("Connection input subscriber must not be null."); 183 | } 184 | this.inputSubscriber = inputSubscriber; 185 | } 186 | 187 | public Subscriber getInputSubscriber() { 188 | return inputSubscriber; 189 | } 190 | 191 | void init(ChannelHandlerContext ctx) { 192 | try { 193 | inputSubscriber.onSubscribe(new Subscription() { 194 | @Override 195 | public void request(long n) { 196 | /*if(n == Long.MAX_VALUE){ 197 | ctx.channel().config().setAutoRead(true); 198 | }*/ 199 | //ctx.read(); implements backpressure 200 | ctx.channel().config().setAutoRead(true); 201 | } 202 | 203 | @Override 204 | public void cancel() { 205 | //implements close on cancel (must be after any pending onComplete) 206 | } 207 | }); 208 | } catch (Throwable error) { 209 | inputSubscriber.onError(error); 210 | } 211 | } 212 | } 213 | } 214 | -------------------------------------------------------------------------------- /ripc-transport-netty4-examples/src/main/java/io/ripc/transport/netty4/tcp/CodecSample.java: -------------------------------------------------------------------------------- 1 | package io.ripc.transport.netty4.tcp; 2 | 3 | import com.fasterxml.jackson.databind.ObjectMapper; 4 | import io.netty.buffer.ByteBuf; 5 | import io.netty.channel.Channel; 6 | import io.netty.channel.ChannelConfig; 7 | import io.netty.channel.ChannelDuplexHandler; 8 | import io.netty.channel.ChannelHandlerContext; 9 | import io.netty.channel.ChannelInitializer; 10 | import io.netty.channel.ChannelOption; 11 | import io.netty.channel.ChannelPromise; 12 | import io.netty.channel.FixedRecvByteBufAllocator; 13 | import io.netty.handler.codec.LineBasedFrameDecoder; 14 | import io.netty.handler.codec.json.JsonObjectDecoder; 15 | import io.netty.handler.codec.string.StringDecoder; 16 | import io.netty.handler.codec.string.StringEncoder; 17 | import io.netty.util.CharsetUtil; 18 | import io.ripc.test.Publishers; 19 | import io.ripc.protocol.tcp.TcpServer; 20 | import org.reactivestreams.Subscriber; 21 | import org.reactivestreams.Subscription; 22 | import org.slf4j.Logger; 23 | import org.slf4j.LoggerFactory; 24 | 25 | import java.nio.charset.Charset; 26 | 27 | public class CodecSample { 28 | 29 | private static final Logger LOG = LoggerFactory.getLogger(CodecSample.class); 30 | 31 | 32 | public static void main(String... args) { 33 | echoWithLineBasedFrameDecoder(); 34 | // echoJsonStreamDecoding(); 35 | } 36 | 37 | private static void echoWithLineBasedFrameDecoder() { 38 | 39 | TcpServer server = Netty4TcpServer.create( 40 | 0, 41 | new ChannelInitializer() { 42 | @Override 43 | protected void initChannel(Channel channel) throws Exception { 44 | int bufferSize = 1024; 45 | ChannelConfig config = channel.config(); 46 | config.setOption(ChannelOption.SO_RCVBUF, bufferSize); 47 | config.setOption(ChannelOption.RCVBUF_ALLOCATOR, new FixedRecvByteBufAllocator(bufferSize)); 48 | channel.pipeline().addFirst( 49 | new LineBasedFrameDecoder(256), 50 | new StringDecoder(CharsetUtil.UTF_8), 51 | new StringEncoder(CharsetUtil.UTF_8)); 52 | } 53 | }); 54 | 55 | server.start(conn -> { 56 | conn.subscribe(new Subscriber() { 57 | 58 | private Subscription subscription; 59 | 60 | @Override 61 | public void onSubscribe(Subscription s) { 62 | this.subscription = s; 63 | LOG.info("requesting 1..."); 64 | s.request(1); 65 | } 66 | 67 | @Override 68 | public void onNext(String s) { 69 | LOG.info("onNext: {}", s); 70 | conn.write(Publishers.just("Hello " + s + "\n")) 71 | .subscribe(new Subscriber() { 72 | 73 | @Override 74 | public void onSubscribe(Subscription s) { 75 | } 76 | 77 | @Override 78 | public void onNext(Void aVoid) { 79 | } 80 | 81 | @Override 82 | public void onError(Throwable t) { 83 | } 84 | 85 | @Override 86 | public void onComplete() { 87 | LOG.info("requesting 1..."); 88 | subscription.request(1); 89 | } 90 | }); 91 | } 92 | 93 | @Override 94 | public void onError(Throwable t) { 95 | LOG.error(t.getMessage(), t); 96 | } 97 | 98 | @Override 99 | public void onComplete() { 100 | LOG.info("onComplete"); 101 | } 102 | }); 103 | 104 | return Publishers.just(null); 105 | }); 106 | 107 | server.awaitShutdown(); 108 | } 109 | 110 | private static void echoJsonStreamDecoding() { 111 | 112 | TcpServer server = Netty4TcpServer.create( 113 | 0, 114 | new ChannelInitializer() { 115 | @Override 116 | protected void initChannel(Channel channel) throws Exception { 117 | channel.pipeline() 118 | .addFirst(new JsonObjectDecoder(), 119 | new JacksonJsonCodec()); 120 | } 121 | }); 122 | 123 | server.start(conn -> { 124 | conn.subscribe(new Subscriber() { 125 | 126 | private Subscription subscription; 127 | 128 | @Override 129 | public void onSubscribe(Subscription s) { 130 | this.subscription = s; 131 | LOG.info("requesting 1..."); 132 | s.request(1); 133 | } 134 | 135 | @Override 136 | public void onNext(Person p) { 137 | LOG.info("onNext: {}", p); 138 | conn.write(Publishers.just(new Person(p.getLastName(), p.getFirstName()))) 139 | .subscribe(new Subscriber() { 140 | 141 | @Override 142 | public void onSubscribe(Subscription s) { 143 | } 144 | 145 | @Override 146 | public void onNext(Void aVoid) { 147 | } 148 | 149 | @Override 150 | public void onError(Throwable t) { 151 | } 152 | 153 | @Override 154 | public void onComplete() { 155 | LOG.info("requesting 1..."); 156 | subscription.request(1); 157 | } 158 | }); 159 | } 160 | 161 | @Override 162 | public void onError(Throwable t) { 163 | LOG.error(t.getMessage(), t); 164 | } 165 | 166 | @Override 167 | public void onComplete() { 168 | LOG.info("onComplete"); 169 | } 170 | }); 171 | 172 | return Publishers.just(null); 173 | }); 174 | 175 | server.awaitShutdown(); 176 | } 177 | 178 | private static class JacksonJsonCodec extends ChannelDuplexHandler { 179 | 180 | private final ObjectMapper mapper = new ObjectMapper(); 181 | 182 | @Override 183 | public void channelRead(ChannelHandlerContext context, Object message) throws Exception { 184 | if (message instanceof ByteBuf) { 185 | Charset charset = Charset.defaultCharset(); 186 | message = this.mapper.readValue(((ByteBuf) message).toString(charset), Person.class); 187 | } 188 | super.channelRead(context, message); 189 | } 190 | 191 | @Override 192 | public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { 193 | if (msg instanceof Person) { 194 | byte[] buff = mapper.writeValueAsBytes(msg); 195 | ByteBuf bb = ctx.alloc().buffer(buff.length); 196 | bb.writeBytes(buff); 197 | msg = bb; 198 | } 199 | super.write(ctx, msg, promise); 200 | } 201 | } 202 | 203 | private static class Person { 204 | 205 | private String firstName; 206 | 207 | private String lastName; 208 | 209 | public Person() { 210 | } 211 | 212 | public Person(String firstName, String lastName) { 213 | this.firstName = firstName; 214 | this.lastName = lastName; 215 | } 216 | 217 | public String getFirstName() { 218 | return firstName; 219 | } 220 | 221 | public Person setFirstName(String firstName) { 222 | this.firstName = firstName; 223 | return this; 224 | } 225 | 226 | public String getLastName() { 227 | return lastName; 228 | } 229 | 230 | public Person setLastName(String lastName) { 231 | this.lastName = lastName; 232 | return this; 233 | } 234 | } 235 | 236 | } 237 | -------------------------------------------------------------------------------- /ripc-transport-netty4-examples/src/main/java/io/netty/handler/codec/json/JsonObjectDecoder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2014 The Netty Project 3 | * 4 | * The Netty Project licenses this file to you under the Apache License, 5 | * version 2.0 (the "License"); you may not use this file except in compliance 6 | * with the License. You may obtain a copy of the License at: 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT 12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the 13 | * License for the specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | 17 | package io.netty.handler.codec.json; 18 | 19 | import io.netty.buffer.ByteBuf; 20 | import io.netty.buffer.ByteBufUtil; 21 | import io.netty.channel.ChannelHandlerContext; 22 | import io.netty.handler.codec.ByteToMessageDecoder; 23 | import io.netty.handler.codec.CorruptedFrameException; 24 | import io.netty.handler.codec.TooLongFrameException; 25 | 26 | import java.util.List; 27 | 28 | /** 29 | * NOTE: This class was copied from Netty 4.1 sources. 30 | * 31 | * Splits a byte stream of JSON objects and arrays into individual objects/arrays and passes them up the 32 | * {@link io.netty.channel.ChannelPipeline}. 33 | * 34 | * This class does not do any real parsing or validation. A sequence of bytes is considered a JSON object/array 35 | * if it contains a matching number of opening and closing braces/brackets. It's up to a subsequent 36 | * {@link io.netty.channel.ChannelHandler} to parse the JSON text into a more usable form i.e. a POJO. 37 | */ 38 | public class JsonObjectDecoder extends ByteToMessageDecoder { 39 | 40 | private static final int ST_CORRUPTED = -1; 41 | private static final int ST_INIT = 0; 42 | private static final int ST_DECODING_NORMAL = 1; 43 | private static final int ST_DECODING_ARRAY_STREAM = 2; 44 | 45 | private int openBraces; 46 | private int idx; 47 | 48 | private int state; 49 | private boolean insideString; 50 | 51 | private final int maxObjectLength; 52 | private final boolean streamArrayElements; 53 | 54 | public JsonObjectDecoder() { 55 | // 1 MB 56 | this(1024 * 1024); 57 | } 58 | 59 | public JsonObjectDecoder(int maxObjectLength) { 60 | this(maxObjectLength, false); 61 | } 62 | 63 | public JsonObjectDecoder(boolean streamArrayElements) { 64 | this(1024 * 1024, streamArrayElements); 65 | } 66 | 67 | /** 68 | * @param maxObjectLength maximum number of bytes a JSON object/array may use (including braces and all). 69 | * Objects exceeding this length are dropped and an {@link io.netty.handler.codec.TooLongFrameException} 70 | * is thrown. 71 | * @param streamArrayElements if set to true and the "top level" JSON object is an array, each of its entries 72 | * is passed through the pipeline individually and immediately after it was fully 73 | * received, allowing for arrays with "infinitely" many elements. 74 | * 75 | */ 76 | public JsonObjectDecoder(int maxObjectLength, boolean streamArrayElements) { 77 | if (maxObjectLength < 1) { 78 | throw new IllegalArgumentException("maxObjectLength must be a positive int"); 79 | } 80 | this.maxObjectLength = maxObjectLength; 81 | this.streamArrayElements = streamArrayElements; 82 | } 83 | 84 | @Override 85 | protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) throws Exception { 86 | if (state == ST_CORRUPTED) { 87 | in.skipBytes(in.readableBytes()); 88 | return; 89 | } 90 | 91 | // index of next byte to process. 92 | int idx = this.idx; 93 | int wrtIdx = in.writerIndex(); 94 | 95 | if (wrtIdx > maxObjectLength) { 96 | // buffer size exceeded maxObjectLength; discarding the complete buffer. 97 | in.skipBytes(in.readableBytes()); 98 | reset(); 99 | throw new TooLongFrameException( 100 | "object length exceeds " + maxObjectLength + ": " + wrtIdx + " bytes discarded"); 101 | } 102 | 103 | for (/* use current idx */; idx < wrtIdx; idx++) { 104 | byte c = in.getByte(idx); 105 | if (state == ST_DECODING_NORMAL) { 106 | decodeByte(c, in, idx); 107 | 108 | // All opening braces/brackets have been closed. That's enough to conclude 109 | // that the JSON object/array is complete. 110 | if (openBraces == 0) { 111 | ByteBuf json = extractObject(ctx, in, in.readerIndex(), idx + 1 - in.readerIndex()); 112 | if (json != null) { 113 | out.add(json); 114 | } 115 | 116 | // The JSON object/array was extracted => discard the bytes from 117 | // the input buffer. 118 | in.readerIndex(idx + 1); 119 | // Reset the object state to get ready for the next JSON object/text 120 | // coming along the byte stream. 121 | reset(); 122 | } 123 | } else if (state == ST_DECODING_ARRAY_STREAM) { 124 | decodeByte(c, in, idx); 125 | 126 | if (!insideString && (openBraces == 1 && c == ',' || openBraces == 0 && c == ']')) { 127 | // skip leading spaces. No range check is needed and the loop will terminate 128 | // because the byte at position idx is not a whitespace. 129 | for (int i = in.readerIndex(); Character.isWhitespace(in.getByte(i)); i++) { 130 | in.skipBytes(1); 131 | } 132 | 133 | // skip trailing spaces. 134 | int idxNoSpaces = idx - 1; 135 | while (idxNoSpaces >= in.readerIndex() && Character.isWhitespace(in.getByte(idxNoSpaces))) { 136 | idxNoSpaces--; 137 | } 138 | 139 | ByteBuf json = extractObject(ctx, in, in.readerIndex(), idxNoSpaces + 1 - in.readerIndex()); 140 | if (json != null) { 141 | out.add(json); 142 | } 143 | 144 | in.readerIndex(idx + 1); 145 | 146 | if (c == ']') { 147 | reset(); 148 | } 149 | } 150 | // JSON object/array detected. Accumulate bytes until all braces/brackets are closed. 151 | } else if (c == '{' || c == '[') { 152 | initDecoding(c); 153 | 154 | if (state == ST_DECODING_ARRAY_STREAM) { 155 | // Discard the array bracket 156 | in.skipBytes(1); 157 | } 158 | // Discard leading spaces in front of a JSON object/array. 159 | } else if (Character.isWhitespace(c)) { 160 | in.skipBytes(1); 161 | } else { 162 | state = ST_CORRUPTED; 163 | throw new CorruptedFrameException( 164 | "invalid JSON received at byte position " + idx + ": " + ByteBufUtil.hexDump(in)); 165 | } 166 | } 167 | 168 | if (in.readableBytes() == 0) { 169 | this.idx = 0; 170 | } else { 171 | this.idx = idx; 172 | } 173 | } 174 | 175 | /** 176 | * Override this method if you want to filter the json objects/arrays that get passed through the pipeline. 177 | */ 178 | @SuppressWarnings("UnusedParameters") 179 | protected ByteBuf extractObject(ChannelHandlerContext ctx, ByteBuf buffer, int index, int length) { 180 | return buffer.slice(index, length).retain(); 181 | } 182 | 183 | private void decodeByte(byte c, ByteBuf in, int idx) { 184 | if ((c == '{' || c == '[') && !insideString) { 185 | openBraces++; 186 | } else if ((c == '}' || c == ']') && !insideString) { 187 | openBraces--; 188 | } else if (c == '"') { 189 | // start of a new JSON string. It's necessary to detect strings as they may 190 | // also contain braces/brackets and that could lead to incorrect results. 191 | if (!insideString) { 192 | insideString = true; 193 | // If the double quote wasn't escaped then this is the end of a string. 194 | } else if (in.getByte(idx - 1) != '\\') { 195 | insideString = false; 196 | } 197 | } 198 | } 199 | 200 | private void initDecoding(byte openingBrace) { 201 | openBraces = 1; 202 | if (openingBrace == '[' && streamArrayElements) { 203 | state = ST_DECODING_ARRAY_STREAM; 204 | } else { 205 | state = ST_DECODING_NORMAL; 206 | } 207 | } 208 | 209 | private void reset() { 210 | insideString = false; 211 | state = ST_INIT; 212 | openBraces = 0; 213 | } 214 | } -------------------------------------------------------------------------------- /ripc-reactor-examples/src/main/java/io/netty/handler/codec/json/JsonObjectDecoder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2014 The Netty Project 3 | * 4 | * The Netty Project licenses this file to you under the Apache License, 5 | * version 2.0 (the "License"); you may not use this file except in compliance 6 | * with the License. You may obtain a copy of the License at: 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT 12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the 13 | * License for the specific language governing permissions and limitations 14 | * under the License. 15 | */ 16 | 17 | package io.netty.handler.codec.json; 18 | 19 | import io.netty.buffer.ByteBuf; 20 | import io.netty.buffer.ByteBufUtil; 21 | import io.netty.channel.ChannelHandlerContext; 22 | import io.netty.handler.codec.ByteToMessageDecoder; 23 | import io.netty.channel.ChannelHandler; 24 | import io.netty.handler.codec.CorruptedFrameException; 25 | import io.netty.handler.codec.TooLongFrameException; 26 | import io.netty.channel.ChannelPipeline; 27 | 28 | import java.util.List; 29 | 30 | /** 31 | * NOTE: This class was copied from Netty 4.1 sources. 32 | * 33 | * Splits a byte stream of JSON objects and arrays into individual objects/arrays and passes them up the 34 | * {@link ChannelPipeline}. 35 | * 36 | * This class does not do any real parsing or validation. A sequence of bytes is considered a JSON object/array 37 | * if it contains a matching number of opening and closing braces/brackets. It's up to a subsequent 38 | * {@link ChannelHandler} to parse the JSON text into a more usable form i.e. a POJO. 39 | */ 40 | public class JsonObjectDecoder extends ByteToMessageDecoder { 41 | 42 | private static final int ST_CORRUPTED = -1; 43 | private static final int ST_INIT = 0; 44 | private static final int ST_DECODING_NORMAL = 1; 45 | private static final int ST_DECODING_ARRAY_STREAM = 2; 46 | 47 | private int openBraces; 48 | private int idx; 49 | 50 | private int state; 51 | private boolean insideString; 52 | 53 | private final int maxObjectLength; 54 | private final boolean streamArrayElements; 55 | 56 | public JsonObjectDecoder() { 57 | // 1 MB 58 | this(1024 * 1024); 59 | } 60 | 61 | public JsonObjectDecoder(int maxObjectLength) { 62 | this(maxObjectLength, false); 63 | } 64 | 65 | public JsonObjectDecoder(boolean streamArrayElements) { 66 | this(1024 * 1024, streamArrayElements); 67 | } 68 | 69 | /** 70 | * @param maxObjectLength maximum number of bytes a JSON object/array may use (including braces and all). 71 | * Objects exceeding this length are dropped and an {@link TooLongFrameException} 72 | * is thrown. 73 | * @param streamArrayElements if set to true and the "top level" JSON object is an array, each of its entries 74 | * is passed through the pipeline individually and immediately after it was fully 75 | * received, allowing for arrays with "infinitely" many elements. 76 | * 77 | */ 78 | public JsonObjectDecoder(int maxObjectLength, boolean streamArrayElements) { 79 | if (maxObjectLength < 1) { 80 | throw new IllegalArgumentException("maxObjectLength must be a positive int"); 81 | } 82 | this.maxObjectLength = maxObjectLength; 83 | this.streamArrayElements = streamArrayElements; 84 | } 85 | 86 | @Override 87 | protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) throws Exception { 88 | if (state == ST_CORRUPTED) { 89 | in.skipBytes(in.readableBytes()); 90 | return; 91 | } 92 | 93 | // index of next byte to process. 94 | int idx = this.idx; 95 | int wrtIdx = in.writerIndex(); 96 | 97 | if (wrtIdx > maxObjectLength) { 98 | // buffer size exceeded maxObjectLength; discarding the complete buffer. 99 | in.skipBytes(in.readableBytes()); 100 | reset(); 101 | throw new TooLongFrameException( 102 | "object length exceeds " + maxObjectLength + ": " + wrtIdx + " bytes discarded"); 103 | } 104 | 105 | for (/* use current idx */; idx < wrtIdx; idx++) { 106 | byte c = in.getByte(idx); 107 | if (state == ST_DECODING_NORMAL) { 108 | decodeByte(c, in, idx); 109 | 110 | // All opening braces/brackets have been closed. That's enough to conclude 111 | // that the JSON object/array is complete. 112 | if (openBraces == 0) { 113 | ByteBuf json = extractObject(ctx, in, in.readerIndex(), idx + 1 - in.readerIndex()); 114 | if (json != null) { 115 | out.add(json); 116 | } 117 | 118 | // The JSON object/array was extracted => discard the bytes from 119 | // the input buffer. 120 | in.readerIndex(idx + 1); 121 | // Reset the object state to get ready for the next JSON object/text 122 | // coming along the byte stream. 123 | reset(); 124 | } 125 | } else if (state == ST_DECODING_ARRAY_STREAM) { 126 | decodeByte(c, in, idx); 127 | 128 | if (!insideString && (openBraces == 1 && c == ',' || openBraces == 0 && c == ']')) { 129 | // skip leading spaces. No range check is needed and the loop will terminate 130 | // because the byte at position idx is not a whitespace. 131 | for (int i = in.readerIndex(); Character.isWhitespace(in.getByte(i)); i++) { 132 | in.skipBytes(1); 133 | } 134 | 135 | // skip trailing spaces. 136 | int idxNoSpaces = idx - 1; 137 | while (idxNoSpaces >= in.readerIndex() && Character.isWhitespace(in.getByte(idxNoSpaces))) { 138 | idxNoSpaces--; 139 | } 140 | 141 | ByteBuf json = extractObject(ctx, in, in.readerIndex(), idxNoSpaces + 1 - in.readerIndex()); 142 | if (json != null) { 143 | out.add(json); 144 | } 145 | 146 | in.readerIndex(idx + 1); 147 | 148 | if (c == ']') { 149 | reset(); 150 | } 151 | } 152 | // JSON object/array detected. Accumulate bytes until all braces/brackets are closed. 153 | } else if (c == '{' || c == '[') { 154 | initDecoding(c); 155 | 156 | if (state == ST_DECODING_ARRAY_STREAM) { 157 | // Discard the array bracket 158 | in.skipBytes(1); 159 | } 160 | // Discard leading spaces in front of a JSON object/array. 161 | } else if (Character.isWhitespace(c)) { 162 | in.skipBytes(1); 163 | } else { 164 | state = ST_CORRUPTED; 165 | throw new CorruptedFrameException( 166 | "invalid JSON received at byte position " + idx + ": " + ByteBufUtil.hexDump(in)); 167 | } 168 | } 169 | 170 | if (in.readableBytes() == 0) { 171 | this.idx = 0; 172 | } else { 173 | this.idx = idx; 174 | } 175 | } 176 | 177 | /** 178 | * Override this method if you want to filter the json objects/arrays that get passed through the pipeline. 179 | */ 180 | @SuppressWarnings("UnusedParameters") 181 | protected ByteBuf extractObject(ChannelHandlerContext ctx, ByteBuf buffer, int index, int length) { 182 | return buffer.slice(index, length).retain(); 183 | } 184 | 185 | private void decodeByte(byte c, ByteBuf in, int idx) { 186 | if ((c == '{' || c == '[') && !insideString) { 187 | openBraces++; 188 | } else if ((c == '}' || c == ']') && !insideString) { 189 | openBraces--; 190 | } else if (c == '"') { 191 | // start of a new JSON string. It's necessary to detect strings as they may 192 | // also contain braces/brackets and that could lead to incorrect results. 193 | if (!insideString) { 194 | insideString = true; 195 | // If the double quote wasn't escaped then this is the end of a string. 196 | } else if (in.getByte(idx - 1) != '\\') { 197 | insideString = false; 198 | } 199 | } 200 | } 201 | 202 | private void initDecoding(byte openingBrace) { 203 | openBraces = 1; 204 | if (openingBrace == '[' && streamArrayElements) { 205 | state = ST_DECODING_ARRAY_STREAM; 206 | } else { 207 | state = ST_DECODING_NORMAL; 208 | } 209 | } 210 | 211 | private void reset() { 212 | insideString = false; 213 | state = ST_INIT; 214 | openBraces = 0; 215 | } 216 | } -------------------------------------------------------------------------------- /LICENSE.txt: -------------------------------------------------------------------------------- 1 | 2 | Apache License 3 | Version 2.0, January 2004 4 | http://www.apache.org/licenses/ 5 | 6 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 7 | 8 | 1. Definitions. 9 | 10 | "License" shall mean the terms and conditions for use, reproduction, 11 | and distribution as defined by Sections 1 through 9 of this document. 12 | 13 | "Licensor" shall mean the copyright owner or entity authorized by 14 | the copyright owner that is granting the License. 15 | 16 | "Legal Entity" shall mean the union of the acting entity and all 17 | other entities that control, are controlled by, or are under common 18 | control with that entity. For the purposes of this definition, 19 | "control" means (i) the power, direct or indirect, to cause the 20 | direction or management of such entity, whether by contract or 21 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 22 | outstanding shares, or (iii) beneficial ownership of such entity. 23 | 24 | "You" (or "Your") shall mean an individual or Legal Entity 25 | exercising permissions granted by this License. 26 | 27 | "Source" form shall mean the preferred form for making modifications, 28 | including but not limited to software source code, documentation 29 | source, and configuration files. 30 | 31 | "Object" form shall mean any form resulting from mechanical 32 | transformation or translation of a Source form, including but 33 | not limited to compiled object code, generated documentation, 34 | and conversions to other media types. 35 | 36 | "Work" shall mean the work of authorship, whether in Source or 37 | Object form, made available under the License, as indicated by a 38 | copyright notice that is included in or attached to the work 39 | (an example is provided in the Appendix below). 40 | 41 | "Derivative Works" shall mean any work, whether in Source or Object 42 | form, that is based on (or derived from) the Work and for which the 43 | editorial revisions, annotations, elaborations, or other modifications 44 | represent, as a whole, an original work of authorship. For the purposes 45 | of this License, Derivative Works shall not include works that remain 46 | separable from, or merely link (or bind by name) to the interfaces of, 47 | the Work and Derivative Works thereof. 48 | 49 | "Contribution" shall mean any work of authorship, including 50 | the original version of the Work and any modifications or additions 51 | to that Work or Derivative Works thereof, that is intentionally 52 | submitted to Licensor for inclusion in the Work by the copyright owner 53 | or by an individual or Legal Entity authorized to submit on behalf of 54 | the copyright owner. For the purposes of this definition, "submitted" 55 | means any form of electronic, verbal, or written communication sent 56 | to the Licensor or its representatives, including but not limited to 57 | communication on electronic mailing lists, source code control systems, 58 | and issue tracking systems that are managed by, or on behalf of, the 59 | Licensor for the purpose of discussing and improving the Work, but 60 | excluding communication that is conspicuously marked or otherwise 61 | designated in writing by the copyright owner as "Not a Contribution." 62 | 63 | "Contributor" shall mean Licensor and any individual or Legal Entity 64 | on behalf of whom a Contribution has been received by Licensor and 65 | subsequently incorporated within the Work. 66 | 67 | 2. Grant of Copyright License. Subject to the terms and conditions of 68 | this License, each Contributor hereby grants to You a perpetual, 69 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 70 | copyright license to reproduce, prepare Derivative Works of, 71 | publicly display, publicly perform, sublicense, and distribute the 72 | Work and such Derivative Works in Source or Object form. 73 | 74 | 3. Grant of Patent License. Subject to the terms and conditions of 75 | this License, each Contributor hereby grants to You a perpetual, 76 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 77 | (except as stated in this section) patent license to make, have made, 78 | use, offer to sell, sell, import, and otherwise transfer the Work, 79 | where such license applies only to those patent claims licensable 80 | by such Contributor that are necessarily infringed by their 81 | Contribution(s) alone or by combination of their Contribution(s) 82 | with the Work to which such Contribution(s) was submitted. If You 83 | institute patent litigation against any entity (including a 84 | cross-claim or counterclaim in a lawsuit) alleging that the Work 85 | or a Contribution incorporated within the Work constitutes direct 86 | or contributory patent infringement, then any patent licenses 87 | granted to You under this License for that Work shall terminate 88 | as of the date such litigation is filed. 89 | 90 | 4. Redistribution. You may reproduce and distribute copies of the 91 | Work or Derivative Works thereof in any medium, with or without 92 | modifications, and in Source or Object form, provided that You 93 | meet the following conditions: 94 | 95 | (a) You must give any other recipients of the Work or 96 | Derivative Works a copy of this License; and 97 | 98 | (b) You must cause any modified files to carry prominent notices 99 | stating that You changed the files; and 100 | 101 | (c) You must retain, in the Source form of any Derivative Works 102 | that You distribute, all copyright, patent, trademark, and 103 | attribution notices from the Source form of the Work, 104 | excluding those notices that do not pertain to any part of 105 | the Derivative Works; and 106 | 107 | (d) If the Work includes a "NOTICE" text file as part of its 108 | distribution, then any Derivative Works that You distribute must 109 | include a readable copy of the attribution notices contained 110 | within such NOTICE file, excluding those notices that do not 111 | pertain to any part of the Derivative Works, in at least one 112 | of the following places: within a NOTICE text file distributed 113 | as part of the Derivative Works; within the Source form or 114 | documentation, if provided along with the Derivative Works; or, 115 | within a display generated by the Derivative Works, if and 116 | wherever such third-party notices normally appear. The contents 117 | of the NOTICE file are for informational purposes only and 118 | do not modify the License. You may add Your own attribution 119 | notices within Derivative Works that You distribute, alongside 120 | or as an addendum to the NOTICE text from the Work, provided 121 | that such additional attribution notices cannot be construed 122 | as modifying the License. 123 | 124 | You may add Your own copyright statement to Your modifications and 125 | may provide additional or different license terms and conditions 126 | for use, reproduction, or distribution of Your modifications, or 127 | for any such Derivative Works as a whole, provided Your use, 128 | reproduction, and distribution of the Work otherwise complies with 129 | the conditions stated in this License. 130 | 131 | 5. Submission of Contributions. Unless You explicitly state otherwise, 132 | any Contribution intentionally submitted for inclusion in the Work 133 | by You to the Licensor shall be under the terms and conditions of 134 | this License, without any additional terms or conditions. 135 | Notwithstanding the above, nothing herein shall supersede or modify 136 | the terms of any separate license agreement you may have executed 137 | with Licensor regarding such Contributions. 138 | 139 | 6. Trademarks. This License does not grant permission to use the trade 140 | names, trademarks, service marks, or product names of the Licensor, 141 | except as required for reasonable and customary use in describing the 142 | origin of the Work and reproducing the content of the NOTICE file. 143 | 144 | 7. Disclaimer of Warranty. Unless required by applicable law or 145 | agreed to in writing, Licensor provides the Work (and each 146 | Contributor provides its Contributions) on an "AS IS" BASIS, 147 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 148 | implied, including, without limitation, any warranties or conditions 149 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 150 | PARTICULAR PURPOSE. You are solely responsible for determining the 151 | appropriateness of using or redistributing the Work and assume any 152 | risks associated with Your exercise of permissions under this License. 153 | 154 | 8. Limitation of Liability. In no event and under no legal theory, 155 | whether in tort (including negligence), contract, or otherwise, 156 | unless required by applicable law (such as deliberate and grossly 157 | negligent acts) or agreed to in writing, shall any Contributor be 158 | liable to You for damages, including any direct, indirect, special, 159 | incidental, or consequential damages of any character arising as a 160 | result of this License or out of the use or inability to use the 161 | Work (including but not limited to damages for loss of goodwill, 162 | work stoppage, computer failure or malfunction, or any and all 163 | other commercial damages or losses), even if such Contributor 164 | has been advised of the possibility of such damages. 165 | 166 | 9. Accepting Warranty or Additional Liability. While redistributing 167 | the Work or Derivative Works thereof, You may choose to offer, 168 | and charge a fee for, acceptance of support, warranty, indemnity, 169 | or other liability obligations and/or rights consistent with this 170 | License. However, in accepting such obligations, You may act only 171 | on Your own behalf and on Your sole responsibility, not on behalf 172 | of any other Contributor, and only if You agree to indemnify, 173 | defend, and hold each Contributor harmless for any liability 174 | incurred by, or claims asserted against, such Contributor by reason 175 | of your accepting any such warranty or additional liability. 176 | 177 | END OF TERMS AND CONDITIONS 178 | 179 | APPENDIX: How to apply the Apache License to your work. 180 | 181 | To apply the Apache License to your work, attach the following 182 | boilerplate notice, with the fields enclosed by brackets "[]" 183 | replaced with your own identifying information. (Don't include 184 | the brackets!) The text should be enclosed in the appropriate 185 | comment syntax for the file format. We also recommend that a 186 | file or class name and description of purpose be included on the 187 | same "printed page" as the copyright notice for easier 188 | identification within third-party archives. 189 | 190 | Copyright [yyyy] [name of copyright owner] 191 | 192 | Licensed under the Apache License, Version 2.0 (the "License"); 193 | you may not use this file except in compliance with the License. 194 | You may obtain a copy of the License at 195 | 196 | http://www.apache.org/licenses/LICENSE-2.0 197 | 198 | Unless required by applicable law or agreed to in writing, software 199 | distributed under the License is distributed on an "AS IS" BASIS, 200 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 201 | See the License for the specific language governing permissions and 202 | limitations under the License. -------------------------------------------------------------------------------- /ripc-test/src/main/java/io/ripc/test/internal/PublisherFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 Pivotal Software Inc, All Rights Reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package io.ripc.test.internal; 17 | 18 | import org.reactivestreams.Publisher; 19 | import org.reactivestreams.Subscriber; 20 | import org.reactivestreams.Subscription; 21 | 22 | 23 | import java.util.concurrent.atomic.AtomicLongFieldUpdater; 24 | import java.util.function.BiConsumer; 25 | import java.util.function.Consumer; 26 | import java.util.function.Function; 27 | 28 | /** 29 | * Original {@see reactor.core.reactivestreams.PublisherFactory} from {@see http://projectreactor.io}. 30 | * Use JDK 8 constructs. 31 | *

32 | * A Reactive Streams {@link org.reactivestreams.Publisher} factory which callbacks on start, request and shutdown 33 | *

34 | * The Publisher will directly forward all the signals passed to the subscribers and complete when onComplete is called. 35 | *

36 | * Create such publisher with the provided factory, E.g.: 37 | *

 38 |  * {@code
 39 |  * PublisherFactory.create((n, sub) -> {
 40 |  *  for(int i = 0; i < n; i++){
 41 |  *    sub.onNext(i);
 42 |  *  }
 43 |  * }
 44 |  * }
 45 |  * 
46 | * 47 | * @author Stephane Maldini 48 | */ 49 | public final class PublisherFactory { 50 | 51 | /** 52 | * Create a {@link Publisher} reacting on requests with the passed {@link BiConsumer} 53 | * 54 | * @param requestConsumer A {@link BiConsumer} with left argument request and right argument target subscriber 55 | * @param The type of the data sequence 56 | * @return a fresh Reactive Streams publisher ready to be subscribed 57 | */ 58 | public static Publisher create(BiConsumer> requestConsumer) { 59 | return create(requestConsumer, null, null); 60 | } 61 | 62 | /** 63 | * Create a {@link Publisher} reacting on requests with the passed {@link BiConsumer} 64 | * The argument {@code contextFactory} is executed once by new subscriber to generate a context shared by every 65 | * request calls. 66 | * 67 | * @param requestConsumer A {@link BiConsumer} with left argument request and right argument target subscriber 68 | * @param contextFactory A {@link Function} called for every new subscriber returning an immutable context (IO 69 | * connection...) 70 | * @param The type of the data sequence 71 | * @param The type of contextual information to be read by the requestConsumer 72 | * @return a fresh Reactive Streams publisher ready to be subscribed 73 | */ 74 | public static Publisher create(BiConsumer> requestConsumer, 75 | Function, C> contextFactory) { 76 | return create(requestConsumer, contextFactory, null); 77 | } 78 | 79 | 80 | /** 81 | * Create a {@link Publisher} reacting on requests with the passed {@link BiConsumer}. 82 | * The argument {@code contextFactory} is executed once by new subscriber to generate a context shared by every 83 | * request calls. 84 | * The argument {@code shutdownConsumer} is executed once by subscriber termination event (cancel, onComplete, 85 | * onError). 86 | * 87 | * @param requestConsumer A {@link BiConsumer} with left argument request and right argument target subscriber 88 | * @param contextFactory A {@link Function} called once for every new subscriber returning an immutable context 89 | * (IO connection...) 90 | * @param shutdownConsumer A {@link Consumer} called once everytime a subscriber terminates: cancel, onComplete(), 91 | * onError() 92 | * @param The type of the data sequence 93 | * @param The type of contextual information to be read by the requestConsumer 94 | * @return a fresh Reactive Streams publisher ready to be subscribed 95 | */ 96 | public static Publisher create(BiConsumer> requestConsumer, 97 | Function, C> contextFactory, 98 | Consumer shutdownConsumer) { 99 | 100 | return new ReactorPublisher(requestConsumer, contextFactory, shutdownConsumer); 101 | } 102 | 103 | 104 | /** 105 | * Create a {@link Publisher} reacting on each available {@link Subscriber} read derived with the passed {@link 106 | * Consumer}. If a previous request is still running, avoid recursion and extend the previous request iterations. 107 | * 108 | * @param requestConsumer A {@link Consumer} invoked when available read with the target subscriber 109 | * @param The type of the data sequence 110 | * @return a fresh Reactive Streams publisher ready to be subscribed 111 | */ 112 | public static Publisher forEach(Consumer> requestConsumer) { 113 | return forEach(requestConsumer, null, null); 114 | } 115 | 116 | /** 117 | * Create a {@link Publisher} reacting on each available {@link Subscriber} read derived with the passed {@link 118 | * Consumer}. If a previous request is still running, avoid recursion and extend the previous request iterations. 119 | * The argument {@code contextFactory} is executed once by new subscriber to generate a context shared by every 120 | * request calls. 121 | * 122 | * @param requestConsumer A {@link Consumer} invoked when available read with the target subscriber 123 | * @param contextFactory A {@link Function} called for every new subscriber returning an immutable context (IO 124 | * connection...) 125 | * @param The type of the data sequence 126 | * @param The type of contextual information to be read by the requestConsumer 127 | * @return a fresh Reactive Streams publisher ready to be subscribed 128 | */ 129 | public static Publisher forEach(Consumer> requestConsumer, 130 | Function, C> contextFactory) { 131 | return forEach(requestConsumer, contextFactory, null); 132 | } 133 | 134 | 135 | /** 136 | * Create a {@link Publisher} reacting on each available {@link Subscriber} read derived with the passed {@link 137 | * Consumer}. If a previous request is still running, avoid recursion and extend the previous request iterations. 138 | * The argument {@code contextFactory} is executed once by new subscriber to generate a context shared by every 139 | * request calls. 140 | * The argument {@code shutdownConsumer} is executed once by subscriber termination event (cancel, onComplete, 141 | * onError). 142 | * 143 | * @param requestConsumer A {@link Consumer} invoked when available read with the target subscriber 144 | * @param contextFactory A {@link Function} called once for every new subscriber returning an immutable context 145 | * (IO connection...) 146 | * @param shutdownConsumer A {@link Consumer} called once everytime a subscriber terminates: cancel, onComplete(), 147 | * onError() 148 | * @param The type of the data sequence 149 | * @param The type of contextual information to be read by the requestConsumer 150 | * @return a fresh Reactive Streams publisher ready to be subscribed 151 | */ 152 | public static Publisher forEach(final Consumer> requestConsumer, 153 | Function, C> contextFactory, 154 | Consumer shutdownConsumer) { 155 | return new ForEachPublisher(requestConsumer, contextFactory, shutdownConsumer); 156 | } 157 | 158 | private static class ReactorPublisher implements Publisher { 159 | 160 | protected final Function, C> contextFactory; 161 | protected final BiConsumer> requestConsumer; 162 | protected final Consumer shutdownConsumer; 163 | 164 | protected ReactorPublisher(BiConsumer> requestConsumer, 165 | Function, C> contextFactory, 166 | Consumer shutdownConsumer) { 167 | this.requestConsumer = requestConsumer; 168 | this.contextFactory = contextFactory; 169 | this.shutdownConsumer = shutdownConsumer; 170 | } 171 | 172 | @Override 173 | final public void subscribe(final Subscriber subscriber) { 174 | try { 175 | final C context = contextFactory != null ? contextFactory.apply(subscriber) : null; 176 | subscriber.onSubscribe(createSubscription(subscriber, context)); 177 | } catch (PrematureCompleteException pce) { 178 | //IGNORE 179 | } catch (Throwable throwable) { 180 | subscriber.onError(throwable); 181 | } 182 | } 183 | 184 | protected Subscription createSubscription(Subscriber subscriber, C context) { 185 | return new SubscriberProxy<>(subscriber, context, requestConsumer, shutdownConsumer); 186 | } 187 | } 188 | 189 | private static final class ForEachPublisher extends ReactorPublisher { 190 | 191 | final Consumer> forEachConsumer; 192 | 193 | 194 | public ForEachPublisher(Consumer> forEachConsumer, Function, C> contextFactory, Consumer shutdownConsumer) { 196 | super(null, contextFactory, shutdownConsumer); 197 | this.forEachConsumer = forEachConsumer; 198 | } 199 | 200 | @Override 201 | protected Subscription createSubscription(Subscriber subscriber, C context) { 202 | return new SubscriberProxy<>(subscriber, context, new ForEachBiConsumer<>(forEachConsumer), 203 | shutdownConsumer); 204 | } 205 | } 206 | 207 | private final static class SubscriberProxy extends SubscriberWithContext implements Subscription { 208 | 209 | private final BiConsumer> requestConsumer; 210 | private final Consumer shutdownConsumer; 211 | 212 | 213 | public SubscriberProxy(Subscriber subscriber, 214 | C context, 215 | BiConsumer> requestConsumer, 216 | Consumer shutdownConsumer 217 | ) { 218 | super(context, subscriber); 219 | this.requestConsumer = requestConsumer; 220 | this.shutdownConsumer = shutdownConsumer; 221 | } 222 | 223 | @Override 224 | public void request(long n) { 225 | if (isCancelled()) { 226 | return; 227 | } 228 | 229 | if (n <= 0) { 230 | onError(new IllegalArgumentException("Spec. Rule 3.9 - Cannot request a non strictly positive number:" + 231 | " " + n)); 232 | return; 233 | } 234 | 235 | try { 236 | requestConsumer.accept(n, this); 237 | } catch (Throwable t) { 238 | onError(t); 239 | } 240 | } 241 | 242 | @Override 243 | public void cancel() { 244 | if (TERMINAL_UPDATER.compareAndSet(this, 0, 1)) { 245 | doShutdown(); 246 | } 247 | } 248 | 249 | @Override 250 | public void onError(Throwable t) { 251 | if (TERMINAL_UPDATER.compareAndSet(this, 0, 1)) { 252 | doShutdown(); 253 | subscriber.onError(t); 254 | } 255 | } 256 | 257 | @Override 258 | public void onComplete() { 259 | if (TERMINAL_UPDATER.compareAndSet(this, 0, 1)) { 260 | doShutdown(); 261 | try { 262 | subscriber.onComplete(); 263 | } catch (Throwable t) { 264 | subscriber.onError(t); 265 | } 266 | } 267 | } 268 | 269 | private void doShutdown() { 270 | if (shutdownConsumer == null) return; 271 | 272 | try { 273 | shutdownConsumer.accept(context); 274 | } catch (Throwable t) { 275 | subscriber.onError(t); 276 | } 277 | } 278 | 279 | @Override 280 | public void onSubscribe(Subscription s) { 281 | throw new UnsupportedOperationException(" the delegate subscriber is already subscribed"); 282 | } 283 | 284 | @Override 285 | public String toString() { 286 | return context != null ? context.toString() : ("SubscriberProxy{" + 287 | "requestConsumer=" + requestConsumer + 288 | ", shutdownConsumer=" + shutdownConsumer + 289 | '}'); 290 | } 291 | } 292 | 293 | private final static class ForEachBiConsumer implements BiConsumer> { 294 | 295 | private final Consumer> requestConsumer; 296 | 297 | private volatile long pending = 0L; 298 | 299 | private final static AtomicLongFieldUpdater PENDING_UPDATER = 300 | AtomicLongFieldUpdater.newUpdater(ForEachBiConsumer.class, "pending"); 301 | 302 | public ForEachBiConsumer(Consumer> requestConsumer) { 303 | this.requestConsumer = requestConsumer; 304 | } 305 | 306 | @Override 307 | public void accept(Long n, SubscriberWithContext sub) { 308 | 309 | if (pending == Long.MAX_VALUE) { 310 | return; 311 | } 312 | 313 | long demand = n; 314 | long afterAdd; 315 | if (!PENDING_UPDATER.compareAndSet(this, 0L, demand) 316 | && (afterAdd = PENDING_UPDATER.addAndGet(this, demand)) != demand) { 317 | if (afterAdd < 0L) { 318 | if (!PENDING_UPDATER.compareAndSet(this, afterAdd, Long.MAX_VALUE)) { 319 | return; 320 | } 321 | } else { 322 | return; 323 | } 324 | } 325 | 326 | do { 327 | long requestCursor = 0l; 328 | while ((requestCursor++ < demand || demand == Long.MAX_VALUE) && !sub.isCancelled()) { 329 | requestConsumer.accept(sub); 330 | } 331 | } while ((demand = PENDING_UPDATER.addAndGet(this, -demand)) > 0L && !sub.isCancelled()); 332 | 333 | } 334 | } 335 | 336 | public static class PrematureCompleteException extends RuntimeException { 337 | static public final PrematureCompleteException INSTANCE = new PrematureCompleteException(); 338 | 339 | private PrematureCompleteException() { 340 | } 341 | 342 | @Override 343 | public synchronized Throwable fillInStackTrace() { 344 | return this; 345 | } 346 | } 347 | } 348 | --------------------------------------------------------------------------------