Skip to content

Commit

Permalink
Add nio http server transport (#29587)
Browse files Browse the repository at this point in the history
This commit is related to #28898. It adds an nio driven http server
transport. Currently it only supports basic http features. Cors,
pipeling, and read timeouts will need to be added in future PRs.
  • Loading branch information
Tim-Brooks committed May 15, 2018
1 parent 6695d11 commit 99b9ab5
Show file tree
Hide file tree
Showing 45 changed files with 3,800 additions and 731 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,25 +20,13 @@
package org.elasticsearch.nio;

import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.ClosedChannelException;
import java.util.LinkedList;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.BiConsumer;
import java.util.function.Consumer;

public class BytesChannelContext extends SocketChannelContext {

private final ReadConsumer readConsumer;
private final InboundChannelBuffer channelBuffer;
private final LinkedList<BytesWriteOperation> queued = new LinkedList<>();
private final AtomicBoolean isClosing = new AtomicBoolean(false);

public BytesChannelContext(NioSocketChannel channel, SocketSelector selector, Consumer<Exception> exceptionHandler,
ReadConsumer readConsumer, InboundChannelBuffer channelBuffer) {
super(channel, selector, exceptionHandler);
this.readConsumer = readConsumer;
this.channelBuffer = channelBuffer;
ReadWriteHandler handler, InboundChannelBuffer channelBuffer) {
super(channel, selector, exceptionHandler, handler, channelBuffer);
}

@Override
Expand All @@ -56,55 +44,30 @@ public int read() throws IOException {

channelBuffer.incrementIndex(bytesRead);

int bytesConsumed = Integer.MAX_VALUE;
while (bytesConsumed > 0 && channelBuffer.getIndex() > 0) {
bytesConsumed = readConsumer.consumeReads(channelBuffer);
channelBuffer.release(bytesConsumed);
}
handleReadBytes();

return bytesRead;
}

@Override
public void sendMessage(ByteBuffer[] buffers, BiConsumer<Void, Throwable> listener) {
if (isClosing.get()) {
listener.accept(null, new ClosedChannelException());
return;
}

BytesWriteOperation writeOperation = new BytesWriteOperation(this, buffers, listener);
SocketSelector selector = getSelector();
if (selector.isOnCurrentThread() == false) {
selector.queueWrite(writeOperation);
return;
}

selector.queueWriteInChannelBuffer(writeOperation);
}

@Override
public void queueWriteOperation(WriteOperation writeOperation) {
getSelector().assertOnSelectorThread();
queued.add((BytesWriteOperation) writeOperation);
}

@Override
public void flushChannel() throws IOException {
getSelector().assertOnSelectorThread();
int ops = queued.size();
if (ops == 1) {
singleFlush(queued.pop());
} else if (ops > 1) {
multiFlush();
boolean lastOpCompleted = true;
FlushOperation flushOperation;
while (lastOpCompleted && (flushOperation = getPendingFlush()) != null) {
try {
if (singleFlush(flushOperation)) {
currentFlushOperationComplete();
} else {
lastOpCompleted = false;
}
} catch (IOException e) {
currentFlushOperationFailed(e);
throw e;
}
}
}

@Override
public boolean hasQueuedWriteOps() {
getSelector().assertOnSelectorThread();
return queued.isEmpty() == false;
}

@Override
public void closeChannel() {
if (isClosing.compareAndSet(false, true)) {
Expand All @@ -117,51 +80,12 @@ public boolean selectorShouldClose() {
return isPeerClosed() || hasIOException() || isClosing.get();
}

@Override
public void closeFromSelector() throws IOException {
getSelector().assertOnSelectorThread();
if (channel.isOpen()) {
IOException channelCloseException = null;
try {
super.closeFromSelector();
} catch (IOException e) {
channelCloseException = e;
}
// Set to true in order to reject new writes before queuing with selector
isClosing.set(true);
channelBuffer.close();
for (BytesWriteOperation op : queued) {
getSelector().executeFailedListener(op.getListener(), new ClosedChannelException());
}
queued.clear();
if (channelCloseException != null) {
throw channelCloseException;
}
}
}

private void singleFlush(BytesWriteOperation headOp) throws IOException {
try {
int written = flushToChannel(headOp.getBuffersToWrite());
headOp.incrementIndex(written);
} catch (IOException e) {
getSelector().executeFailedListener(headOp.getListener(), e);
throw e;
}

if (headOp.isFullyFlushed()) {
getSelector().executeListener(headOp.getListener(), null);
} else {
queued.push(headOp);
}
}

private void multiFlush() throws IOException {
boolean lastOpCompleted = true;
while (lastOpCompleted && queued.isEmpty() == false) {
BytesWriteOperation op = queued.pop();
singleFlush(op);
lastOpCompleted = op.isFullyFlushed();
}
/**
* Returns a boolean indicating if the operation was fully flushed.
*/
private boolean singleFlush(FlushOperation flushOperation) throws IOException {
int written = flushToChannel(flushOperation.getBuffersToWrite());
flushOperation.incrementIndex(written);
return flushOperation.isFullyFlushed();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.elasticsearch.nio;

import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.List;
import java.util.function.BiConsumer;

public abstract class BytesWriteHandler implements ReadWriteHandler {

private static final List<FlushOperation> EMPTY_LIST = Collections.emptyList();

public WriteOperation createWriteOperation(SocketChannelContext context, Object message, BiConsumer<Void, Throwable> listener) {
assert message instanceof ByteBuffer[] : "This channel only supports messages that are of type: " + ByteBuffer[].class
+ ". Found type: " + message.getClass() + ".";
return new FlushReadyWrite(context, (ByteBuffer[]) message, listener);
}

public List<FlushOperation> writeToBytes(WriteOperation writeOperation) {
assert writeOperation instanceof FlushReadyWrite : "Write operation must be flush ready";
return Collections.singletonList((FlushReadyWrite) writeOperation);
}

public List<FlushOperation> pollFlushOperations() {
return EMPTY_LIST;
}

public void close() {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,17 +23,15 @@
import java.util.Arrays;
import java.util.function.BiConsumer;

public class BytesWriteOperation implements WriteOperation {
public class FlushOperation {

private final SocketChannelContext channelContext;
private final BiConsumer<Void, Throwable> listener;
private final ByteBuffer[] buffers;
private final int[] offsets;
private final int length;
private int internalIndex;

public BytesWriteOperation(SocketChannelContext channelContext, ByteBuffer[] buffers, BiConsumer<Void, Throwable> listener) {
this.channelContext = channelContext;
public FlushOperation(ByteBuffer[] buffers, BiConsumer<Void, Throwable> listener) {
this.listener = listener;
this.buffers = buffers;
this.offsets = new int[buffers.length];
Expand All @@ -46,16 +44,10 @@ public BytesWriteOperation(SocketChannelContext channelContext, ByteBuffer[] buf
length = offset;
}

@Override
public BiConsumer<Void, Throwable> getListener() {
return listener;
}

@Override
public SocketChannelContext getChannel() {
return channelContext;
}

public boolean isFullyFlushed() {
assert length >= internalIndex : "Should never have an index that is greater than the length [length=" + length + ", index="
+ internalIndex + "]";
Expand Down Expand Up @@ -84,5 +76,4 @@ public ByteBuffer[] getBuffersToWrite() {

return postIndexBuffers;
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.elasticsearch.nio;

import java.nio.ByteBuffer;
import java.util.function.BiConsumer;

public class FlushReadyWrite extends FlushOperation implements WriteOperation {

private final SocketChannelContext channelContext;
private final ByteBuffer[] buffers;

FlushReadyWrite(SocketChannelContext channelContext, ByteBuffer[] buffers, BiConsumer<Void, Throwable> listener) {
super(buffers, listener);
this.channelContext = channelContext;
this.buffers = buffers;
}

@Override
public SocketChannelContext getChannel() {
return channelContext;
}

@Override
public ByteBuffer[] getObject() {
return buffers;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.elasticsearch.nio;

import java.io.IOException;
import java.util.List;
import java.util.function.BiConsumer;

/**
* Implements the application specific logic for handling inbound and outbound messages for a channel.
*/
public interface ReadWriteHandler {

/**
* This method is called when a message is queued with a channel. It can be called from any thread.
* This method should validate that the message is a valid type and return a write operation object
* to be queued with the channel
*
* @param context the channel context
* @param message the message
* @param listener the listener to be called when the message is sent
* @return the write operation to be queued
*/
WriteOperation createWriteOperation(SocketChannelContext context, Object message, BiConsumer<Void, Throwable> listener);

/**
* This method is called on the event loop thread. It should serialize a write operation object to bytes
* that can be flushed to the raw nio channel.
*
* @param writeOperation to be converted to bytes
* @return the operations to flush the bytes to the channel
*/
List<FlushOperation> writeToBytes(WriteOperation writeOperation);

/**
* Returns any flush operations that are ready to flush. This exists as a way to check if any flush
* operations were produced during a read call.
*
* @return flush operations
*/
List<FlushOperation> pollFlushOperations();

/**
* This method handles bytes that have been read from the network. It should return the number of bytes
* consumed so that they can be released.
*
* @param channelBuffer of bytes read from the network
* @return the number of bytes consumed
* @throws IOException if an exception occurs
*/
int consumeReads(InboundChannelBuffer channelBuffer) throws IOException;

void close() throws IOException;
}
Loading

0 comments on commit 99b9ab5

Please sign in to comment.