HTTP/2 Non Active Stream RFC Corrections

Motivation:
codec-http2 couples the dependency tree state with the remainder of the stream state (Http2Stream). This makes implementing constraints where stream state and dependency tree state diverge in the RFC challenging. For example the RFC recommends retaining dependency tree state after a stream transitions to closed [1]. Dependency tree state can be exchanged on streams in IDLE. In practice clients may use stream IDs for the purpose of establishing QoS classes and therefore retaining this dependency tree state can be important to client perceived performance. It is difficult to limit the total amount of state we retain when stream state and dependency tree state is combined.

Modifications:
- Remove dependency tree, priority, and weight related items from public facing Http2Connection and Http2Stream APIs. This information is optional to track and depends on the flow controller implementation.
- Move all dependency tree, priority, and weight related code from DefaultHttp2Connection to WeightedFairQueueByteDistributor. This is currently the only place which cares about priority. We can pull out the dependency tree related code in the future if it is generally useful to expose for other implementations.
- DefaultHttp2Connection should explicitly limit the number of reserved streams now that IDLE streams are no longer created.

Result:
More compliant with the HTTP/2 RFC.
Fixes https://github.com/netty/netty/issues/6206.

[1] https://tools.ietf.org/html/rfc7540#section-5.3.4
This commit is contained in:
Scott Mitchell 2017-01-24 12:50:39 -08:00
parent 6e5b25733f
commit 3482651e0c
39 changed files with 2392 additions and 2348 deletions

View File

@ -20,7 +20,9 @@ import io.netty.handler.codec.http2.Http2HeadersEncoder.SensitivityDetector;
import io.netty.util.internal.UnstableApi;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_MAX_RESERVED_STREAMS;
import static io.netty.util.internal.ObjectUtil.checkNotNull;
import static io.netty.util.internal.ObjectUtil.checkPositiveOrZero;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
@ -84,6 +86,7 @@ public abstract class AbstractHttp2ConnectionHandlerBuilder<T extends Http2Conne
// The property that will prohibit connection() and codec() if set by server(),
// because this property is used only when this builder creates a Http2Connection.
private Boolean isServer;
private Integer maxReservedStreams;
// The property that will prohibit server() and codec() if set by connection().
private Http2Connection connection;
@ -170,6 +173,29 @@ public abstract class AbstractHttp2ConnectionHandlerBuilder<T extends Http2Conne
return self();
}
/**
* Get the maximum number of streams which can be in the reserved state at any given time.
* <p>
* By default this value will be ignored on the server for local endpoint. This is because the RFC provides
* no way to explicitly communicate a limit to how many states can be in the reserved state, and instead relies
* on the peer to send RST_STREAM frames when they will be rejected.
*/
protected int maxReservedStreams() {
return maxReservedStreams != null ? maxReservedStreams : DEFAULT_MAX_RESERVED_STREAMS;
}
/**
* Set the maximum number of streams which can be in the reserved state at any given time.
*/
protected B maxReservedStreams(int maxReservedStreams) {
enforceConstraint("server", "connection", connection);
enforceConstraint("server", "codec", decoder);
enforceConstraint("server", "codec", encoder);
this.maxReservedStreams = checkPositiveOrZero(maxReservedStreams, "maxReservedStreams");
return self();
}
/**
* Returns the {@link Http2Connection} to use.
*
@ -183,6 +209,7 @@ public abstract class AbstractHttp2ConnectionHandlerBuilder<T extends Http2Conne
* Sets the {@link Http2Connection} to use.
*/
protected B connection(Http2Connection connection) {
enforceConstraint("connection", "maxReservedStreams", maxReservedStreams);
enforceConstraint("connection", "server", isServer);
enforceConstraint("connection", "codec", decoder);
enforceConstraint("connection", "codec", encoder);
@ -215,6 +242,7 @@ public abstract class AbstractHttp2ConnectionHandlerBuilder<T extends Http2Conne
*/
protected B codec(Http2ConnectionDecoder decoder, Http2ConnectionEncoder encoder) {
enforceConstraint("codec", "server", isServer);
enforceConstraint("codec", "maxReservedStreams", maxReservedStreams);
enforceConstraint("codec", "connection", connection);
enforceConstraint("codec", "frameLogger", frameLogger);
enforceConstraint("codec", "validateHeaders", validateHeaders);
@ -328,7 +356,7 @@ public abstract class AbstractHttp2ConnectionHandlerBuilder<T extends Http2Conne
Http2Connection connection = this.connection;
if (connection == null) {
connection = new DefaultHttp2Connection(isServer());
connection = new DefaultHttp2Connection(isServer(), maxReservedStreams());
}
return buildFromConnection(connection);

View File

@ -18,7 +18,6 @@ package io.netty.handler.codec.http2;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelPromise;
import io.netty.handler.codec.http2.Http2Stream.State;
import io.netty.util.collection.IntCollections;
import io.netty.util.collection.IntObjectHashMap;
import io.netty.util.collection.IntObjectMap;
import io.netty.util.collection.IntObjectMap.PrimitiveEntry;
@ -27,7 +26,6 @@ import io.netty.util.concurrent.Promise;
import io.netty.util.concurrent.UnaryPromiseNotifier;
import io.netty.util.internal.EmptyArrays;
import io.netty.util.internal.PlatformDependent;
import io.netty.util.internal.SystemPropertyUtil;
import io.netty.util.internal.UnstableApi;
import io.netty.util.internal.logging.InternalLogger;
import io.netty.util.internal.logging.InternalLoggerFactory;
@ -42,9 +40,7 @@ import java.util.Queue;
import java.util.Set;
import static io.netty.handler.codec.http2.Http2CodecUtil.CONNECTION_STREAM_ID;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_WEIGHT;
import static io.netty.handler.codec.http2.Http2CodecUtil.MIN_WEIGHT;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_MAX_RESERVED_STREAMS;
import static io.netty.handler.codec.http2.Http2Error.INTERNAL_ERROR;
import static io.netty.handler.codec.http2.Http2Error.PROTOCOL_ERROR;
import static io.netty.handler.codec.http2.Http2Error.REFUSED_STREAM;
@ -59,7 +55,8 @@ import static io.netty.handler.codec.http2.Http2Stream.State.OPEN;
import static io.netty.handler.codec.http2.Http2Stream.State.RESERVED_LOCAL;
import static io.netty.handler.codec.http2.Http2Stream.State.RESERVED_REMOTE;
import static io.netty.util.internal.ObjectUtil.checkNotNull;
import static java.lang.Math.max;
import static io.netty.util.internal.ObjectUtil.checkPositiveOrZero;
import static java.lang.Integer.MAX_VALUE;
/**
* Simple implementation of {@link Http2Connection}.
@ -74,20 +71,13 @@ public class DefaultHttp2Connection implements Http2Connection {
final DefaultEndpoint<Http2LocalFlowController> localEndpoint;
final DefaultEndpoint<Http2RemoteFlowController> remoteEndpoint;
/**
* The initial size of the children map is chosen to be conservative on initial memory allocations under
* the assumption that most streams will have a small number of children. This choice may be
* sub-optimal if when children are present there are many children (i.e. a web page which has many
* dependencies to load).
*
* Visible only for testing!
*/
static final int INITIAL_CHILDREN_MAP_SIZE =
max(1, SystemPropertyUtil.getInt("io.netty.http2.childrenMapSize", 4));
/**
* We chose a {@link List} over a {@link Set} to avoid allocating an {@link Iterator} objects when iterating over
* the listeners.
* <p>
* Initial size of 4 because the default configuration currently has 3 listeners
* (local/remote flow controller and {@link StreamByteDistributor}) and we leave room for 1 extra.
* We could be more aggressive but the ArrayList resize will double the size if we are too small.
*/
final List<Listener> listeners = new ArrayList<Listener>(4);
final ActiveStreams activeStreams;
@ -95,14 +85,26 @@ public class DefaultHttp2Connection implements Http2Connection {
/**
* Creates a new connection with the given settings.
*
* @param server
* whether or not this end-point is the server-side of the HTTP/2 connection.
* @param server whether or not this end-point is the server-side of the HTTP/2 connection.
*/
public DefaultHttp2Connection(boolean server) {
this(server, DEFAULT_MAX_RESERVED_STREAMS);
}
/**
* Creates a new connection with the given settings.
* @param server whether or not this end-point is the server-side of the HTTP/2 connection.
* @param maxReservedStreams The maximum amount of streams which can exist in the reserved state for each endpoint.
*/
public DefaultHttp2Connection(boolean server, int maxReservedStreams) {
activeStreams = new ActiveStreams(listeners);
localEndpoint = new DefaultEndpoint<Http2LocalFlowController>(server);
remoteEndpoint = new DefaultEndpoint<Http2RemoteFlowController>(!server);
// Reserved streams are excluded from the SETTINGS_MAX_CONCURRENT_STREAMS limit according to [1] and the RFC
// doesn't define a way to communicate the limit on reserved streams. We rely upon the peer to send RST_STREAM
// in response to any locally enforced limits being exceeded [2].
// [1] https://tools.ietf.org/html/rfc7540#section-5.1.2
// [2] https://tools.ietf.org/html/rfc7540#section-8.2.2
localEndpoint = new DefaultEndpoint<Http2LocalFlowController>(server, server ? MAX_VALUE : maxReservedStreams);
remoteEndpoint = new DefaultEndpoint<Http2RemoteFlowController>(!server, maxReservedStreams);
// Add the connection stream to the map.
streamMap.put(connectionStream.id(), connectionStream);
@ -293,13 +295,15 @@ public class DefaultHttp2Connection implements Http2Connection {
* used if non-{@code null}.
*/
void removeStream(DefaultStream stream, Iterator<?> itr) {
if (stream.parent().removeChild(stream)) {
if (itr == null) {
streamMap.remove(stream.id());
} else {
itr.remove();
}
final boolean removed;
if (itr == null) {
removed = streamMap.remove(stream.id()) != null;
} else {
itr.remove();
removed = true;
}
if (removed) {
for (int i = 0; i < listeners.size(); i++) {
try {
listeners.get(i).onStreamRemoved(stream);
@ -375,9 +379,6 @@ public class DefaultHttp2Connection implements Http2Connection {
private final int id;
private final PropertyMap properties = new PropertyMap();
private State state;
private short weight = DEFAULT_PRIORITY_WEIGHT;
private DefaultStream parent;
private IntObjectMap<DefaultStream> children = IntCollections.emptyMap();
private byte sentState;
DefaultStream(int id, State state) {
@ -443,87 +444,6 @@ public class DefaultHttp2Connection implements Http2Connection {
return properties.remove(verifyKey(key));
}
@Override
public final boolean isRoot() {
return parent == null;
}
@Override
public final short weight() {
return weight;
}
@Override
public final DefaultStream parent() {
return parent;
}
@Override
public final boolean isDescendantOf(Http2Stream stream) {
Http2Stream next = parent();
while (next != null) {
if (next == stream) {
return true;
}
next = next.parent();
}
return false;
}
@Override
public final boolean isLeaf() {
return numChildren() == 0;
}
@Override
public final int numChildren() {
return children.size();
}
@Override
public Http2Stream forEachChild(Http2StreamVisitor visitor) throws Http2Exception {
for (DefaultStream stream : children.values()) {
if (!visitor.visit(stream)) {
return stream;
}
}
return null;
}
@Override
public Http2Stream setPriority(int parentStreamId, short weight, boolean exclusive) throws Http2Exception {
if (weight < MIN_WEIGHT || weight > MAX_WEIGHT) {
throw new IllegalArgumentException(String.format(
"Invalid weight: %d. Must be between %d and %d (inclusive).", weight, MIN_WEIGHT, MAX_WEIGHT));
}
DefaultStream newParent = (DefaultStream) stream(parentStreamId);
if (newParent == null) {
// Streams can depend on other streams in the IDLE state. We must ensure
// the stream has been "created" in order to use it in the priority tree.
newParent = createdBy().createIdleStream(parentStreamId);
} else if (this == newParent) {
throw new IllegalArgumentException("A stream cannot depend on itself");
}
// Already have a priority. Re-prioritize the stream.
weight(weight);
if (newParent != parent() || (exclusive && newParent.numChildren() != 1)) {
final List<ParentChangedEvent> events;
if (newParent.isDescendantOf(this)) {
events = new ArrayList<ParentChangedEvent>(2 + (exclusive ? newParent.numChildren() : 0));
parent.takeChild(newParent, false, events);
} else {
events = new ArrayList<ParentChangedEvent>(1 + (exclusive ? newParent.numChildren() : 0));
}
newParent.takeChild(this, exclusive, events);
notifyParentChanged(events);
}
return this;
}
@Override
public Http2Stream open(boolean halfClosed) throws Http2Exception {
state = activeState(id, state, isLocal(), halfClosed);
@ -587,16 +507,6 @@ public class DefaultHttp2Connection implements Http2Connection {
return this;
}
private void initChildrenIfEmpty() {
if (children == IntCollections.<DefaultStream>emptyMap()) {
initChildren();
}
}
private void initChildren() {
children = new IntObjectHashMap<DefaultStream>(INITIAL_CHILDREN_MAP_SIZE);
}
DefaultEndpoint<? extends Http2FlowController> createdBy() {
return localEndpoint.isValidStreamId(id) ? localEndpoint : remoteEndpoint;
}
@ -605,101 +515,6 @@ public class DefaultHttp2Connection implements Http2Connection {
return localEndpoint.isValidStreamId(id);
}
final void weight(short weight) {
if (weight != this.weight) {
final short oldWeight = this.weight;
this.weight = weight;
for (int i = 0; i < listeners.size(); i++) {
try {
listeners.get(i).onWeightChanged(this, oldWeight);
} catch (Throwable cause) {
logger.error("Caught Throwable from listener onWeightChanged.", cause);
}
}
}
}
/**
* Remove all children with the exception of {@code streamToRetain}.
* This method is intended to be used to support an exclusive priority dependency operation.
* @return The map of children prior to this operation, excluding {@code streamToRetain} if present.
*/
private IntObjectMap<DefaultStream> removeAllChildrenExcept(DefaultStream streamToRetain) {
streamToRetain = children.remove(streamToRetain.id());
IntObjectMap<DefaultStream> prevChildren = children;
// This map should be re-initialized in anticipation for the 1 exclusive child which will be added.
// It will either be added directly in this method, or after this method is called...but it will be added.
initChildren();
if (streamToRetain != null) {
children.put(streamToRetain.id(), streamToRetain);
}
return prevChildren;
}
/**
* Adds a child to this priority. If exclusive is set, any children of this node are moved to being dependent on
* the child.
*/
final void takeChild(Iterator<PrimitiveEntry<DefaultStream>> childItr, DefaultStream child, boolean exclusive,
List<ParentChangedEvent> events) {
DefaultStream oldParent = child.parent();
if (oldParent != this) {
events.add(new ParentChangedEvent(child, oldParent));
notifyParentChanging(child, this);
child.parent = this;
// If the childItr is not null we are iterating over the oldParent.children collection and should
// use the iterator to remove from the collection to avoid concurrent modification. Otherwise it is
// assumed we are not iterating over this collection and it is safe to call remove directly.
if (childItr != null) {
childItr.remove();
} else if (oldParent != null) {
oldParent.children.remove(child.id());
}
// Lazily initialize the children to save object allocations.
initChildrenIfEmpty();
final Http2Stream oldChild = children.put(child.id(), child);
assert oldChild == null : "A stream with the same stream ID was already in the child map.";
}
if (exclusive && !children.isEmpty()) {
// If it was requested that this child be the exclusive dependency of this node,
// move any previous children to the child node, becoming grand children of this node.
Iterator<PrimitiveEntry<DefaultStream>> itr = removeAllChildrenExcept(child).entries().iterator();
while (itr.hasNext()) {
child.takeChild(itr, itr.next().value(), false, events);
}
}
}
final void takeChild(DefaultStream child, boolean exclusive, List<ParentChangedEvent> events) {
takeChild(null, child, exclusive, events);
}
/**
* Removes the child priority and moves any of its dependencies to being direct dependencies on this node.
*/
final boolean removeChild(DefaultStream child) {
if (children.remove(child.id()) != null) {
List<ParentChangedEvent> events = new ArrayList<ParentChangedEvent>(1 + child.numChildren());
events.add(new ParentChangedEvent(child, child.parent()));
notifyParentChanging(child, null);
child.parent = null;
// Move up any grand children to be directly dependent on this node.
Iterator<PrimitiveEntry<DefaultStream>> itr = child.children.entries().iterator();
while (itr.hasNext()) {
takeChild(itr, itr.next().value(), false, events);
}
notifyParentChanged(events);
return true;
}
return false;
}
/**
* Provides the lazy initialization for the {@link DefaultStream} data map.
*/
@ -740,59 +555,6 @@ public class DefaultHttp2Connection implements Http2Connection {
}
}
/**
* Allows a correlation to be made between a stream and its old parent before a parent change occurs
*/
private static final class ParentChangedEvent {
private final Http2Stream stream;
private final Http2Stream oldParent;
/**
* Create a new instance
* @param stream The stream who has had a parent change
* @param oldParent The previous parent
*/
ParentChangedEvent(Http2Stream stream, Http2Stream oldParent) {
this.stream = stream;
this.oldParent = oldParent;
}
/**
* Notify all listeners of the tree change event
* @param l The listener to notify
*/
public void notifyListener(Listener l) {
try {
l.onPriorityTreeParentChanged(stream, oldParent);
} catch (Throwable cause) {
logger.error("Caught Throwable from listener onPriorityTreeParentChanged.", cause);
}
}
}
/**
* Notify all listeners of the priority tree change events (in ascending order)
* @param events The events (top down order) which have changed
*/
private void notifyParentChanged(List<ParentChangedEvent> events) {
for (int i = 0; i < events.size(); ++i) {
ParentChangedEvent event = events.get(i);
for (int j = 0; j < listeners.size(); j++) {
event.notifyListener(listeners.get(j));
}
}
}
private void notifyParentChanging(Http2Stream stream, Http2Stream newParent) {
for (int i = 0; i < listeners.size(); i++) {
try {
listeners.get(i).onPriorityTreeParentChanging(stream, newParent);
} catch (Throwable cause) {
logger.error("Caught Throwable from listener onPriorityTreeParentChanging.", cause);
}
}
}
/**
* Stream class representing the connection, itself.
*/
@ -816,11 +578,6 @@ public class DefaultHttp2Connection implements Http2Connection {
throw new UnsupportedOperationException();
}
@Override
public Http2Stream setPriority(int parentStreamId, short weight, boolean exclusive) {
throw new UnsupportedOperationException();
}
@Override
public Http2Stream open(boolean halfClosed) {
throw new UnsupportedOperationException();
@ -883,13 +640,14 @@ public class DefaultHttp2Connection implements Http2Connection {
private int lastStreamKnownByPeer = -1;
private boolean pushToAllowed = true;
private F flowController;
private int maxActiveStreams;
private int maxStreams;
private int maxActiveStreams;
private final int maxReservedStreams;
// Fields accessed by inner classes
int numActiveStreams;
int numStreams;
DefaultEndpoint(boolean server) {
DefaultEndpoint(boolean server, int maxReservedStreams) {
this.server = server;
// Determine the starting stream ID for this endpoint. Client-initiated streams
@ -907,7 +665,9 @@ public class DefaultHttp2Connection implements Http2Connection {
// Push is disallowed by default for servers and allowed for clients.
pushToAllowed = !server;
maxStreams = maxActiveStreams = Integer.MAX_VALUE;
maxActiveStreams = MAX_VALUE;
this.maxReservedStreams = checkPositiveOrZero(maxReservedStreams, "maxReservedStreams");
updateMaxStreams();
}
@Override
@ -938,7 +698,10 @@ public class DefaultHttp2Connection implements Http2Connection {
return numActiveStreams < maxActiveStreams;
}
private DefaultStream createStream(int streamId, State state) throws Http2Exception {
@Override
public DefaultStream createStream(int streamId, boolean halfClosed) throws Http2Exception {
State state = activeState(streamId, IDLE, isLocal(), halfClosed);
checkNewStreamAllowed(streamId, state);
// Create and initialize the stream.
@ -947,17 +710,7 @@ public class DefaultHttp2Connection implements Http2Connection {
incrementExpectedStreamId(streamId);
addStream(stream);
return stream;
}
@Override
public DefaultStream createIdleStream(int streamId) throws Http2Exception {
return createStream(streamId, IDLE);
}
@Override
public DefaultStream createStream(int streamId, boolean halfClosed) throws Http2Exception {
DefaultStream stream = createStream(streamId, activeState(streamId, IDLE, isLocal(), halfClosed));
stream.activate();
return stream;
}
@ -981,7 +734,7 @@ public class DefaultHttp2Connection implements Http2Connection {
throw connectionError(PROTOCOL_ERROR, "Stream %d is not open for sending push promise", parent.id());
}
if (!opposite().allowPushTo()) {
throw connectionError(PROTOCOL_ERROR, "Server push not allowed to opposite endpoint.");
throw connectionError(PROTOCOL_ERROR, "Server push not allowed to opposite endpoint");
}
State state = isLocal() ? RESERVED_LOCAL : RESERVED_REMOTE;
checkNewStreamAllowed(streamId, state);
@ -1000,9 +753,6 @@ public class DefaultHttp2Connection implements Http2Connection {
// Add the stream to the map and priority tree.
streamMap.put(stream.id(), stream);
List<ParentChangedEvent> events = new ArrayList<ParentChangedEvent>(1);
connectionStream.takeChild(stream, false, events);
// Notify the listeners of the event.
for (int i = 0; i < listeners.size(); i++) {
try {
@ -1011,8 +761,6 @@ public class DefaultHttp2Connection implements Http2Connection {
logger.error("Caught Throwable from listener onStreamAdded.", cause);
}
}
notifyParentChanged(events);
}
@Override
@ -1039,18 +787,9 @@ public class DefaultHttp2Connection implements Http2Connection {
}
@Override
public int maxStreams() {
return maxStreams;
}
@Override
public void maxStreams(int maxActiveStreams, int maxStreams) throws Http2Exception {
if (maxStreams < maxActiveStreams) {
throw connectionError(PROTOCOL_ERROR, "maxStream[%d] streams must be >= maxActiveStreams[%d]",
maxStreams, maxActiveStreams);
}
this.maxStreams = maxStreams;
public void maxActiveStreams(int maxActiveStreams) {
this.maxActiveStreams = maxActiveStreams;
updateMaxStreams();
}
@Override
@ -1082,16 +821,21 @@ public class DefaultHttp2Connection implements Http2Connection {
return isLocal() ? remoteEndpoint : localEndpoint;
}
private void updateMaxStreams() {
maxStreams = (int) Math.min(MAX_VALUE, (long) maxActiveStreams + maxReservedStreams);
}
private void checkNewStreamAllowed(int streamId, State state) throws Http2Exception {
assert state != IDLE;
if (goAwayReceived() && streamId > localEndpoint.lastStreamKnownByPeer()) {
throw connectionError(PROTOCOL_ERROR, "Cannot create stream %d since this endpoint has received a " +
"GOAWAY frame with last stream id %d.", streamId,
localEndpoint.lastStreamKnownByPeer());
}
if (streamId < 0) {
throw new Http2NoMoreStreamIdsException();
}
if (!isValidStreamId(streamId)) {
if (streamId < 0) {
throw new Http2NoMoreStreamIdsException();
}
throw connectionError(PROTOCOL_ERROR, "Request stream %d is not correct for %s connection", streamId,
server ? "server" : "client");
}
@ -1104,12 +848,9 @@ public class DefaultHttp2Connection implements Http2Connection {
if (nextStreamIdToCreate <= 0) {
throw connectionError(REFUSED_STREAM, "Stream IDs are exhausted for this endpoint.");
}
if (state.localSideOpen() || state.remoteSideOpen()) {
if (!canOpenStream()) {
throw streamError(streamId, REFUSED_STREAM, "Maximum active streams violated for this endpoint.");
}
} else if (numStreams == maxStreams) {
throw streamError(streamId, REFUSED_STREAM, "Maximum streams violated for this endpoint.");
boolean isReserved = state == RESERVED_LOCAL || state == RESERVED_REMOTE;
if (!isReserved && !canOpenStream() || isReserved && numStreams >= maxStreams) {
throw streamError(streamId, REFUSED_STREAM, "Maximum active streams violated for this endpoint.");
}
if (isClosed()) {
throw connectionError(INTERNAL_ERROR, "Attempted to create stream id %d after connection was closed",
@ -1174,14 +915,6 @@ public class DefaultHttp2Connection implements Http2Connection {
pendingEvents.add(new Event() {
@Override
public void process() {
// When deactivate is called the stream state has already been set to CLOSE however
// it is possible that since this job has been queued other circumstances have caused
// it to be removed from the priority tree and thus have a null parent (i.e. reprioritization).
// If the parent is null this means it has already been removed from active streams and we
// should not process the removal any further as this will lead to a NPE.
if (stream.parent() == null) {
return;
}
removeFromActiveStreams(stream, itr);
}
});
@ -1274,6 +1007,11 @@ public class DefaultHttp2Connection implements Http2Connection {
* A registry of all stream property keys known by this connection.
*/
private final class PropertyKeyRegistry {
/**
* Initial size of 4 because the default configuration currently has 3 listeners
* (local/remote flow controller and {@link StreamByteDistributor}) and we leave room for 1 extra.
* We could be more aggressive but the ArrayList resize will double the size if we are too small.
*/
final List<DefaultPropertyKey> keys = new ArrayList<DefaultPropertyKey>(4);
/**

View File

@ -17,7 +17,6 @@ package io.netty.handler.codec.http2;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelHandlerContext;
import io.netty.handler.codec.http2.Http2Connection.Endpoint;
import io.netty.handler.codec.http2.Http2Exception.ClosedStreamCreationException;
import io.netty.util.internal.UnstableApi;
import io.netty.util.internal.logging.InternalLogger;
import io.netty.util.internal.logging.InternalLoggerFactory;
@ -25,7 +24,6 @@ import io.netty.util.internal.logging.InternalLoggerFactory;
import java.util.List;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT;
import static io.netty.handler.codec.http2.Http2CodecUtil.SMALLEST_MAX_CONCURRENT_STREAMS;
import static io.netty.handler.codec.http2.Http2Error.INTERNAL_ERROR;
import static io.netty.handler.codec.http2.Http2Error.PROTOCOL_ERROR;
import static io.netty.handler.codec.http2.Http2Error.STREAM_CLOSED;
@ -307,14 +305,7 @@ public class DefaultHttp2ConnectionDecoder implements Http2ConnectionDecoder {
stream.state());
}
try {
// This call will create a stream for streamDependency if necessary.
// For this reason it must be done before notifying the listener.
stream.setPriority(streamDependency, weight, exclusive);
} catch (ClosedStreamCreationException ignored) {
// It is possible that either the stream for this frame or the parent stream is closed.
// In this case we should ignore the exception and allow the frame to be sent.
}
encoder.flowController().updateDependencyTree(streamId, streamDependency, weight, exclusive);
listener.onHeadersRead(ctx, streamId, headers, streamDependency, weight, exclusive, padding, endOfStream);
@ -327,33 +318,7 @@ public class DefaultHttp2ConnectionDecoder implements Http2ConnectionDecoder {
@Override
public void onPriorityRead(ChannelHandlerContext ctx, int streamId, int streamDependency, short weight,
boolean exclusive) throws Http2Exception {
Http2Stream stream = connection.stream(streamId);
try {
if (stream == null) {
if (connection.streamMayHaveExisted(streamId)) {
logger.info("{} ignoring PRIORITY frame for stream {}. Stream doesn't exist but may " +
" have existed", ctx.channel(), streamId);
return;
}
// PRIORITY frames always identify a stream. This means that if a PRIORITY frame is the
// first frame to be received for a stream that we must create the stream.
stream = connection.remote().createIdleStream(streamId);
} else if (streamCreatedAfterGoAwaySent(streamId)) {
logger.info("{} ignoring PRIORITY frame for stream {}. Stream created after GOAWAY sent. " +
"Last known stream by peer {}",
ctx.channel(), streamId, connection.remote().lastStreamKnownByPeer());
return;
}
// This call will create a stream for streamDependency if necessary.
// For this reason it must be done before notifying the listener.
stream.setPriority(streamDependency, weight, exclusive);
} catch (ClosedStreamCreationException ignored) {
// It is possible that either the stream for this frame or the parent stream is closed.
// In this case we should ignore the exception and allow the frame to be sent.
}
encoder.flowController().updateDependencyTree(streamId, streamDependency, weight, exclusive);
listener.onPriorityRead(ctx, streamId, streamDependency, weight, exclusive);
}
@ -411,8 +376,7 @@ public class DefaultHttp2ConnectionDecoder implements Http2ConnectionDecoder {
Long maxConcurrentStreams = settings.maxConcurrentStreams();
if (maxConcurrentStreams != null) {
int value = (int) min(maxConcurrentStreams, MAX_VALUE);
connection.remote().maxStreams(value, calculateMaxStreams(value));
connection.remote().maxActiveStreams((int) min(maxConcurrentStreams, MAX_VALUE));
}
Long headerTableSize = settings.headerTableSize();
@ -436,17 +400,6 @@ public class DefaultHttp2ConnectionDecoder implements Http2ConnectionDecoder {
}
}
/**
* Calculate the {@code maxStreams} parameter for the {@link Endpoint#maxStreams(int, int)} method based upon
* {@code SETTINGS_MAX_CONCURRENT_STREAMS}.
* @param maxConcurrentStreams {@code SETTINGS_MAX_CONCURRENT_STREAMS}.
* @return the {@code maxStreams} parameter for the {@link Endpoint#maxStreams(int, int)} method.
*/
private int calculateMaxStreams(int maxConcurrentStreams) {
int maxStreams = maxConcurrentStreams + SMALLEST_MAX_CONCURRENT_STREAMS;
return maxStreams < 0 ? MAX_VALUE : maxStreams;
}
@Override
public void onSettingsRead(ChannelHandlerContext ctx, Http2Settings settings) throws Http2Exception {
encoder.remoteSettings(settings);

View File

@ -21,7 +21,6 @@ import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelPromise;
import io.netty.channel.CoalescingBufferQueue;
import io.netty.handler.codec.http2.Http2Exception.ClosedStreamCreationException;
import io.netty.util.internal.UnstableApi;
import java.util.ArrayDeque;
@ -89,8 +88,7 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder {
Long maxConcurrentStreams = settings.maxConcurrentStreams();
if (maxConcurrentStreams != null) {
// TODO(scott): define an extension setting so we can communicate/enforce the maxStreams limit locally.
connection.local().maxStreams((int) min(maxConcurrentStreams, MAX_VALUE), MAX_VALUE);
connection.local().maxActiveStreams((int) min(maxConcurrentStreams, MAX_VALUE));
}
Long headerTableSize = settings.headerTableSize();
@ -202,7 +200,7 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder {
// Pass headers to the flow-controller so it can maintain their sequence relative to DATA frames.
flowController.addFlowControlled(stream,
new FlowControlledHeaders(stream, headers, streamDependency, weight, exclusive, padding,
endOfStream, promise));
true, promise));
return promise;
}
} catch (Throwable t) {
@ -215,23 +213,6 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder {
@Override
public ChannelFuture writePriority(ChannelHandlerContext ctx, int streamId, int streamDependency, short weight,
boolean exclusive, ChannelPromise promise) {
try {
// Update the priority on this stream.
Http2Stream stream = connection.stream(streamId);
if (stream == null) {
stream = connection.local().createIdleStream(streamId);
}
// The set priority operation must be done before sending the frame. The parent may not yet exist
// and the priority tree may also be modified before sending.
stream.setPriority(streamDependency, weight, exclusive);
} catch (ClosedStreamCreationException ignored) {
// It is possible that either the stream for this frame or the parent stream is closed.
// In this case we should ignore the exception and allow the frame to be sent.
} catch (Throwable t) {
return promise.setFailure(t);
}
return frameWriter.writePriority(ctx, streamId, streamDependency, weight, exclusive, promise);
}
@ -495,7 +476,7 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder {
protected boolean endOfStream;
protected int padding;
protected FlowControlledBase(final Http2Stream stream, int padding, boolean endOfStream,
FlowControlledBase(final Http2Stream stream, int padding, boolean endOfStream,
final ChannelPromise promise) {
if (padding < 0) {
throw new IllegalArgumentException("padding must be >= 0");

View File

@ -176,6 +176,11 @@ public class DefaultHttp2RemoteFlowController implements Http2RemoteFlowControll
monitor.channelWritabilityChange();
}
@Override
public void updateDependencyTree(int childStreamId, int parentStreamId, short weight, boolean exclusive) {
streamByteDistributor.updateDependencyTree(childStreamId, parentStreamId, weight, exclusive);
}
private boolean isChannelWritable() {
return ctx != null && isChannelWritable0();
}

View File

@ -111,6 +111,13 @@ public final class Http2CodecUtil {
*/
public static final long DEFAULT_HEADER_LIST_SIZE = 8192;
public static final int DEFAULT_MAX_FRAME_SIZE = MAX_FRAME_SIZE_LOWER_BOUND;
/**
* The assumed minimum value for {@code SETTINGS_MAX_CONCURRENT_STREAMS} as
* recommended by the <a herf="https://tools.ietf.org/html/rfc7540#section-6.5.2">HTTP/2 spec</a>.
*/
public static final int SMALLEST_MAX_CONCURRENT_STREAMS = 100;
static final int DEFAULT_MAX_RESERVED_STREAMS = SMALLEST_MAX_CONCURRENT_STREAMS;
static final int DEFAULT_MIN_ALLOCATION_CHUNK = 1024;
/**
* Calculate the threshold in bytes which should trigger a {@code GO_AWAY} if a set of headers exceeds this amount.
@ -142,12 +149,6 @@ public final class Http2CodecUtil {
return streamId >= 0;
}
/**
* The assumed minimum value for {@code SETTINGS_MAX_CONCURRENT_STREAMS} as
* recommended by the HTTP/2 spec.
*/
public static final int SMALLEST_MAX_CONCURRENT_STREAMS = 100;
/**
* Indicates whether or not the given value for max frame size falls within the valid range.
*/

View File

@ -76,41 +76,6 @@ public interface Http2Connection {
*/
void onStreamRemoved(Http2Stream stream);
/**
* Notifies the listener that a priority tree parent change has occurred. This method will be invoked
* in a top down order relative to the priority tree. This method will also be invoked after all tree
* structure changes have been made and the tree is in steady state relative to the priority change
* which caused the tree structure to change.
* <p>
* If a {@link RuntimeException} is thrown it will be logged and <strong>not propagated</strong>.
* Throwing from this method is not supported and is considered a programming error.
* @param stream The stream which had a parent change (new parent and children will be steady state)
* @param oldParent The old parent which {@code stream} used to be a child of (may be {@code null})
*/
void onPriorityTreeParentChanged(Http2Stream stream, Http2Stream oldParent);
/**
* Notifies the listener that a parent dependency is about to change
* This is called while the tree is being restructured and so the tree
* structure is not necessarily steady state.
* <p>
* If a {@link RuntimeException} is thrown it will be logged and <strong>not propagated</strong>.
* Throwing from this method is not supported and is considered a programming error.
* @param stream The stream which the parent is about to change to {@code newParent}
* @param newParent The stream which will be the parent of {@code stream}
*/
void onPriorityTreeParentChanging(Http2Stream stream, Http2Stream newParent);
/**
* Notifies the listener that the weight has changed for {@code stream}.
* <p>
* If a {@link RuntimeException} is thrown it will be logged and <strong>not propagated</strong>.
* Throwing from this method is not supported and is considered a programming error.
* @param stream The stream which the weight has changed
* @param oldWeight The old weight for {@code stream}
*/
void onWeightChanged(Http2Stream stream, short oldWeight);
/**
* Called when a {@code GOAWAY} frame was sent for the connection.
* <p>
@ -171,25 +136,6 @@ public interface Http2Connection {
*/
boolean canOpenStream();
/**
* Creates a stream initiated by this endpoint. This could fail for the following reasons:
* <ul>
* <li>The requested stream ID is not the next sequential ID for this endpoint.</li>
* <li>The stream already exists.</li>
* <li>The connection is marked as going away.</li>
* </ul>
* <p>
* Note that IDLE streams can always be created so long as there are stream IDs available.
* The {@link #numActiveStreams()} will be enforced upon attempting to open the stream.
* <p>
* If the stream is intended to initialized to {@link Http2Stream.State#OPEN} then use
* {@link #createStream(int, boolean)} otherwise optimizations in {@link Listener}s may not work
* and memory may be thrashed. The caller is expected to {@link Http2Stream#open(boolean)} the stream.
* @param streamId The ID of the stream
* @see Http2Stream#open(boolean)
*/
Http2Stream createIdleStream(int streamId) throws Http2Exception;
/**
* Creates a stream initiated by this endpoint. This could fail for the following reasons:
* <ul>
@ -199,9 +145,8 @@ public interface Http2Connection {
* <li>The connection is marked as going away.</li>
* </ul>
* <p>
* This method differs from {@link #createIdleStream(int)} because the initial state of the stream will be
* Immediately set before notifying {@link Listener}s. The state transition is sensitive to {@code halfClosed}
* and is defined by {@link Http2Stream#open(boolean)}.
* The initial state of the stream will be immediately set before notifying {@link Listener}s. The state
* transition is sensitive to {@code halfClosed} and is defined by {@link Http2Stream#open(boolean)}.
* @param streamId The ID of the stream
* @param halfClosed see {@link Http2Stream#open(boolean)}.
* @see Http2Stream#open(boolean)
@ -262,23 +207,13 @@ public interface Http2Connection {
int maxActiveStreams();
/**
* The limit imposed by {@link #maxActiveStreams()} does not apply to streams in the IDLE state. Since IDLE
* streams can still consume resources this limit will include streams in all states.
* @return The maximum number of streams that can exist at any given time.
*/
int maxStreams();
/**
* Sets the limit for {@code SETTINGS_MAX_CONCURRENT_STREAMS} and the limit for {@link #maxStreams()}.
* Sets the limit for {@code SETTINGS_MAX_CONCURRENT_STREAMS}.
* @param maxActiveStreams The maximum number of streams (created by this endpoint) that are allowed to be
* active at once. This is the
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_CONCURRENT_STREAMS</a> value sent
* from the opposite endpoint to restrict stream creation by this endpoint.
* @param maxStreams The limit imposed by {@link #maxActiveStreams()} does not apply to streams in the IDLE
* state. Since IDLE streams can still consume resources this limit will include streams in all states.
* @throws Http2Exception if {@code maxStreams < maxActiveStream}.
*/
void maxStreams(int maxActiveStreams, int maxStreams) throws Http2Exception;
void maxActiveStreams(int maxActiveStreams);
/**
* Gets the ID of the stream last successfully created by this endpoint.

View File

@ -22,7 +22,6 @@ import io.netty.util.internal.UnstableApi;
*/
@UnstableApi
public class Http2ConnectionAdapter implements Http2Connection.Listener {
@Override
public void onStreamAdded(Http2Stream stream) {
}
@ -50,16 +49,4 @@ public class Http2ConnectionAdapter implements Http2Connection.Listener {
@Override
public void onGoAwayReceived(int lastStreamId, long errorCode, ByteBuf debugData) {
}
@Override
public void onPriorityTreeParentChanged(Http2Stream stream, Http2Stream oldParent) {
}
@Override
public void onPriorityTreeParentChanging(Http2Stream stream, Http2Stream newParent) {
}
@Override
public void onWeightChanged(Http2Stream stream, short oldWeight) {
}
}

View File

@ -105,18 +105,6 @@ public class Http2EventAdapter implements Http2Connection.Listener, Http2FrameLi
public void onStreamRemoved(Http2Stream stream) {
}
@Override
public void onPriorityTreeParentChanged(Http2Stream stream, Http2Stream oldParent) {
}
@Override
public void onPriorityTreeParentChanging(Http2Stream stream, Http2Stream newParent) {
}
@Override
public void onWeightChanged(Http2Stream stream, short oldWeight) {
}
@Override
public void onGoAwaySent(int lastStreamId, long errorCode, ByteBuf debugData) {
}

View File

@ -133,6 +133,7 @@ public class Http2FrameCodec extends ChannelDuplexHandler {
// Visible for testing
Http2FrameCodec(boolean server, Http2FrameWriter frameWriter, Http2FrameLogger frameLogger,
Http2Settings initialSettings) {
// TODO(scott): configure maxReservedStreams when API is more finalized.
Http2Connection connection = new DefaultHttp2Connection(server);
frameWriter = new Http2OutboundFrameLogger(frameWriter, frameLogger);
Http2ConnectionEncoder encoder = new DefaultHttp2ConnectionEncoder(connection, frameWriter);

View File

@ -81,6 +81,17 @@ public interface Http2RemoteFlowController extends Http2FlowController {
*/
void channelWritabilityChanged() throws Http2Exception;
/**
* Explicitly update the dependency tree. This method is called independently of stream state changes.
* @param childStreamId The stream identifier associated with the child stream.
* @param parentStreamId The stream identifier associated with the parent stream. May be {@code 0},
* to make {@code childStreamId} and immediate child of the connection.
* @param weight The weight which is used relative to other child streams for {@code parentStreamId}. This value
* must be between 1 and 256 (inclusive).
* @param exclusive If {@code childStreamId} should be the exclusive dependency of {@code parentStreamId}.
*/
void updateDependencyTree(int childStreamId, int parentStreamId, short weight, boolean exclusive);
/**
* Implementations of this interface are used to progressively write chunks of the underlying
* payload to the stream. A payload is considered to be fully written if {@link #write} has

View File

@ -129,61 +129,6 @@ public interface Http2Stream {
*/
<V> V removeProperty(Http2Connection.PropertyKey key);
/**
* Updates an priority for this stream. Calling this method may affect the structure of the
* priority tree.
*
* @param parentStreamId the parent stream that given stream should depend on. May be {@code 0},
* if the stream has no dependencies and should be an immediate child of the
* connection.
* @param weight the weight to be assigned to this stream relative to its parent. This value
* must be between 1 and 256 (inclusive)
* @param exclusive indicates that the stream should be the exclusive dependent on its parent.
* This only applies if the stream has a parent.
* @return this stream.
*/
Http2Stream setPriority(int parentStreamId, short weight, boolean exclusive) throws Http2Exception;
/**
* Indicates whether or not this stream is the root node of the priority tree.
*/
boolean isRoot();
/**
* Indicates whether or not this is a leaf node (i.e. {@link #numChildren} is 0) of the priority tree.
*/
boolean isLeaf();
/**
* Returns weight assigned to the dependency with the parent. The weight will be a value
* between 1 and 256.
*/
short weight();
/**
* The parent (i.e. the node in the priority tree on which this node depends), or {@code null}
* if this is the root node (i.e. the connection, itself).
*/
Http2Stream parent();
/**
* Indicates whether or not this stream is a descendant in the priority tree from the given stream.
*/
boolean isDescendantOf(Http2Stream stream);
/**
* Returns the number of child streams directly dependent on this stream.
*/
int numChildren();
/**
* Provide a means of iterating over the children of this stream.
*
* @param visitor The visitor which will visit each child stream.
* @return The stream before iteration stopped or {@code null} if iteration went past the end.
*/
Http2Stream forEachChild(Http2StreamVisitor visitor) throws Http2Exception;
/**
* Indicates that headers has been sent to the remote on this stream.
*/

View File

@ -340,7 +340,6 @@ public final class HttpConversionUtil {
* The following headers are only used if they can not be found in from the {@code HOST} header or the
* {@code Request-Line} as defined by <a href="https://tools.ietf.org/html/rfc7230">rfc7230</a>
* <ul>
* <li>{@link ExtensionHeaderNames#AUTHORITY}</li>
* <li>{@link ExtensionHeaderNames#SCHEME}</li>
* </ul>
* {@link ExtensionHeaderNames#PATH} is ignored and instead extracted from the {@code Request-Line}.

View File

@ -1,213 +0,0 @@
/*
* Copyright 2014 The Netty Project
*
* The Netty Project 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 io.netty.handler.codec.http2;
import io.netty.channel.ChannelHandlerContext;
import io.netty.handler.codec.http.DefaultHttpHeaders;
import io.netty.handler.codec.http.FullHttpMessage;
import io.netty.handler.codec.http.HttpHeaders;
import io.netty.util.AsciiString;
import io.netty.util.internal.UnstableApi;
import java.util.Iterator;
import java.util.Map.Entry;
import static io.netty.handler.codec.http2.Http2Error.PROTOCOL_ERROR;
import static io.netty.handler.codec.http2.Http2Exception.connectionError;
/**
* Translate header/data/priority HTTP/2 frame events into HTTP events. Just as {@link InboundHttp2ToHttpAdapter}
* may generate multiple {@link FullHttpMessage} objects per stream, this class is more likely to
* generate multiple messages per stream because the chances of an HTTP/2 event happening outside
* the header/data message flow is more likely.
*/
@UnstableApi
public final class InboundHttp2ToHttpPriorityAdapter extends InboundHttp2ToHttpAdapter {
private static final AsciiString OUT_OF_MESSAGE_SEQUENCE_METHOD = new AsciiString(
HttpConversionUtil.OUT_OF_MESSAGE_SEQUENCE_METHOD.toString());
private static final AsciiString OUT_OF_MESSAGE_SEQUENCE_PATH = new AsciiString(
HttpConversionUtil.OUT_OF_MESSAGE_SEQUENCE_PATH);
private static final AsciiString OUT_OF_MESSAGE_SEQUENCE_RETURN_CODE = new AsciiString(
HttpConversionUtil.OUT_OF_MESSAGE_SEQUENCE_RETURN_CODE.toString());
private final Http2Connection.PropertyKey outOfMessageFlowHeadersKey;
InboundHttp2ToHttpPriorityAdapter(Http2Connection connection, int maxContentLength,
boolean validateHttpHeaders,
boolean propagateSettings) {
super(connection, maxContentLength, validateHttpHeaders, propagateSettings);
outOfMessageFlowHeadersKey = connection.newKey();
}
private HttpHeaders getOutOfMessageFlowHeaders(Http2Stream stream) {
return stream.getProperty(outOfMessageFlowHeadersKey);
}
private void putOutOfMessageFlowHeaders(Http2Stream stream, HttpHeaders headers) {
stream.setProperty(outOfMessageFlowHeadersKey, headers);
}
private HttpHeaders removeOutOfMessageFlowHeaders(Http2Stream stream) {
return stream.removeProperty(outOfMessageFlowHeadersKey);
}
/**
* Get either the header or the trailing headers depending on which is valid to add to
* @param msg The message containing the headers and trailing headers
* @return The headers object which can be appended to or modified
*/
private static HttpHeaders getActiveHeaders(FullHttpMessage msg) {
return msg.content().isReadable() ? msg.trailingHeaders() : msg.headers();
}
/**
* This method will add the {@code headers} to the out of order headers map
* @param stream The stream associated with {@code headers}
* @param headers Newly encountered out of order headers which must be stored for future use
*/
private void importOutOfMessageFlowHeaders(Http2Stream stream, HttpHeaders headers) {
final HttpHeaders outOfMessageFlowHeader = getOutOfMessageFlowHeaders(stream);
if (outOfMessageFlowHeader == null) {
putOutOfMessageFlowHeaders(stream, headers);
} else {
outOfMessageFlowHeader.setAll(headers);
}
}
/**
* Take any saved out of order headers and export them to {@code headers}
* @param stream The stream to search for out of order headers for
* @param headers If any out of order headers exist for {@code stream} they will be added to this object
*/
private void exportOutOfMessageFlowHeaders(Http2Stream stream, final HttpHeaders headers) {
final HttpHeaders outOfMessageFlowHeader = getOutOfMessageFlowHeaders(stream);
if (outOfMessageFlowHeader != null) {
headers.setAll(outOfMessageFlowHeader);
}
}
/**
* This will remove all headers which are related to priority tree events
* @param headers The headers to remove the priority tree elements from
*/
private static void removePriorityRelatedHeaders(HttpHeaders headers) {
headers.remove(HttpConversionUtil.ExtensionHeaderNames.STREAM_DEPENDENCY_ID.text());
headers.remove(HttpConversionUtil.ExtensionHeaderNames.STREAM_WEIGHT.text());
}
/**
* Initializes the pseudo header fields for out of message flow HTTP/2 headers
* @param headers The headers to be initialized with pseudo header values
*/
private void initializePseudoHeaders(Http2Headers headers) {
if (connection.isServer()) {
headers.method(OUT_OF_MESSAGE_SEQUENCE_METHOD).path(OUT_OF_MESSAGE_SEQUENCE_PATH);
} else {
headers.status(OUT_OF_MESSAGE_SEQUENCE_RETURN_CODE);
}
}
/**
* Add all the HTTP headers into the HTTP/2 headers object
* @param httpHeaders The HTTP headers to translate to HTTP/2
* @param http2Headers The target HTTP/2 headers
*/
private static void addHttpHeadersToHttp2Headers(HttpHeaders httpHeaders, final Http2Headers http2Headers) {
Iterator<Entry<CharSequence, CharSequence>> iter = httpHeaders.iteratorCharSequence();
while (iter.hasNext()) {
Entry<CharSequence, CharSequence> entry = iter.next();
http2Headers.add(AsciiString.of(entry.getKey()), AsciiString.of(entry.getValue()));
}
}
@Override
protected void fireChannelRead(ChannelHandlerContext ctx, FullHttpMessage msg, boolean release,
Http2Stream stream) {
exportOutOfMessageFlowHeaders(stream, getActiveHeaders(msg));
super.fireChannelRead(ctx, msg, release, stream);
}
@Override
protected FullHttpMessage processHeadersBegin(ChannelHandlerContext ctx, Http2Stream stream, Http2Headers headers,
boolean endOfStream, boolean allowAppend, boolean appendToTrailer) throws Http2Exception {
FullHttpMessage msg = super.processHeadersBegin(ctx, stream, headers,
endOfStream, allowAppend, appendToTrailer);
if (msg != null) {
exportOutOfMessageFlowHeaders(stream, getActiveHeaders(msg));
}
return msg;
}
@Override
public void onPriorityTreeParentChanged(Http2Stream stream, Http2Stream oldParent) {
Http2Stream parent = stream.parent();
FullHttpMessage msg = getMessage(stream);
if (msg == null) {
// msg may be null if a HTTP/2 frame event is received outside the HTTP message flow
// For example a PRIORITY frame can be received in any state but the HTTP message flow
// takes place while the stream is OPEN.
if (parent != null && !parent.equals(connection.connectionStream())) {
HttpHeaders headers = new DefaultHttpHeaders();
headers.setInt(HttpConversionUtil.ExtensionHeaderNames.STREAM_DEPENDENCY_ID.text(), parent.id());
importOutOfMessageFlowHeaders(stream, headers);
}
} else {
if (parent == null) {
removePriorityRelatedHeaders(msg.headers());
removePriorityRelatedHeaders(msg.trailingHeaders());
} else if (!parent.equals(connection.connectionStream())) {
HttpHeaders headers = getActiveHeaders(msg);
headers.setInt(HttpConversionUtil.ExtensionHeaderNames.STREAM_DEPENDENCY_ID.text(), parent.id());
}
}
}
@Override
public void onWeightChanged(Http2Stream stream, short oldWeight) {
FullHttpMessage msg = getMessage(stream);
final HttpHeaders headers;
if (msg == null) {
// msg may be null if a HTTP/2 frame event in received outside the HTTP message flow
// For example a PRIORITY frame can be received in any state
// and the HTTP message flow exists in OPEN.
headers = new DefaultHttpHeaders();
importOutOfMessageFlowHeaders(stream, headers);
} else {
headers = getActiveHeaders(msg);
}
headers.setShort(HttpConversionUtil.ExtensionHeaderNames.STREAM_WEIGHT.text(), stream.weight());
}
@Override
public void onPriorityRead(ChannelHandlerContext ctx, int streamId, int streamDependency, short weight,
boolean exclusive) throws Http2Exception {
Http2Stream stream = connection.stream(streamId);
if (stream == null) {
return;
}
FullHttpMessage msg = getMessage(stream);
if (msg == null) {
HttpHeaders httpHeaders = removeOutOfMessageFlowHeaders(stream);
if (httpHeaders == null) {
throw connectionError(PROTOCOL_ERROR, "Priority Frame recieved for unknown stream id %d", streamId);
}
Http2Headers http2Headers = new DefaultHttp2Headers(validateHttpHeaders, httpHeaders.size());
initializePseudoHeaders(http2Headers);
addHttpHeadersToHttp2Headers(httpHeaders, http2Headers);
msg = newMessage(stream, http2Headers, validateHttpHeaders, ctx.alloc());
fireChannelRead(ctx, msg, false, stream);
}
}
}

View File

@ -1,67 +0,0 @@
/*
* Copyright 2015 The Netty Project
*
* The Netty Project 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 io.netty.handler.codec.http2;
import io.netty.util.internal.UnstableApi;
/**
* Builds an {@link InboundHttp2ToHttpPriorityAdapter}.
*/
@UnstableApi
public final class InboundHttp2ToHttpPriorityAdapterBuilder
extends AbstractInboundHttp2ToHttpAdapterBuilder<InboundHttp2ToHttpPriorityAdapter,
InboundHttp2ToHttpPriorityAdapterBuilder> {
/**
* Creates a new {@link InboundHttp2ToHttpPriorityAdapter} builder for the specified
* {@link Http2Connection}.
*
* @param connection the object which will provide connection notification events
* for the current connection
*/
public InboundHttp2ToHttpPriorityAdapterBuilder(Http2Connection connection) {
super(connection);
}
@Override
public InboundHttp2ToHttpPriorityAdapterBuilder maxContentLength(int maxContentLength) {
return super.maxContentLength(maxContentLength);
}
@Override
public InboundHttp2ToHttpPriorityAdapterBuilder validateHttpHeaders(boolean validate) {
return super.validateHttpHeaders(validate);
}
@Override
public InboundHttp2ToHttpPriorityAdapterBuilder propagateSettings(boolean propagate) {
return super.propagateSettings(propagate);
}
@Override
public InboundHttp2ToHttpPriorityAdapter build() {
return super.build();
}
@Override
protected InboundHttp2ToHttpPriorityAdapter build(Http2Connection connection,
int maxContentLength,
boolean validateHttpHeaders,
boolean propagateSettings) throws Exception {
return new InboundHttp2ToHttpPriorityAdapter(connection, maxContentLength,
validateHttpHeaders, propagateSettings);
}
}

View File

@ -81,6 +81,17 @@ public interface StreamByteDistributor {
*/
void updateStreamableBytes(StreamState state);
/**
* Explicitly update the dependency tree. This method is called independently of stream state changes.
* @param childStreamId The stream identifier associated with the child stream.
* @param parentStreamId The stream identifier associated with the parent stream. May be {@code 0},
* to make {@code childStreamId} and immediate child of the connection.
* @param weight The weight which is used relative to other child streams for {@code parentStreamId}. This value
* must be between 1 and 256 (inclusive).
* @param exclusive If {@code childStreamId} should be the exclusive dependency of {@code parentStreamId}.
*/
void updateDependencyTree(int childStreamId, int parentStreamId, short weight, boolean exclusive);
/**
* Distributes up to {@code maxBytes} to those streams containing streamable bytes and
* iterates across those streams to write the appropriate bytes. Criteria for

View File

@ -19,6 +19,7 @@ import io.netty.util.internal.UnstableApi;
import java.util.ArrayDeque;
import java.util.Deque;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_MIN_ALLOCATION_CHUNK;
import static io.netty.handler.codec.http2.Http2CodecUtil.streamableBytes;
import static io.netty.handler.codec.http2.Http2Error.INTERNAL_ERROR;
import static io.netty.handler.codec.http2.Http2Exception.connectionError;
@ -34,8 +35,6 @@ import static java.lang.Math.min;
*/
@UnstableApi
public final class UniformStreamByteDistributor implements StreamByteDistributor {
static final int DEFAULT_MIN_ALLOCATION_CHUNK = 1024;
private final Http2Connection.PropertyKey stateKey;
private final Deque<State> queue = new ArrayDeque<State>(4);
@ -87,9 +86,12 @@ public final class UniformStreamByteDistributor implements StreamByteDistributor
}
@Override
public boolean distribute(int maxBytes, Writer writer) throws Http2Exception {
checkNotNull(writer, "writer");
public void updateDependencyTree(int childStreamId, int parentStreamId, short weight, boolean exclusive) {
// This class ignores priority and dependency!
}
@Override
public boolean distribute(int maxBytes, Writer writer) throws Http2Exception {
final int size = queue.size();
if (size == 0) {
return totalStreamableBytes > 0;

View File

@ -14,17 +14,32 @@
*/
package io.netty.handler.codec.http2;
import io.netty.util.collection.IntCollections;
import io.netty.util.collection.IntObjectHashMap;
import io.netty.util.collection.IntObjectMap;
import io.netty.util.internal.DefaultPriorityQueue;
import io.netty.util.internal.EmptyPriorityQueue;
import io.netty.util.internal.MathUtil;
import io.netty.util.internal.PriorityQueue;
import io.netty.util.internal.PriorityQueueNode;
import io.netty.util.internal.SystemPropertyUtil;
import io.netty.util.internal.UnstableApi;
import java.util.Queue;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import static io.netty.handler.codec.http2.Http2CodecUtil.CONNECTION_STREAM_ID;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_MIN_ALLOCATION_CHUNK;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_WEIGHT;
import static io.netty.handler.codec.http2.Http2CodecUtil.MIN_WEIGHT;
import static io.netty.handler.codec.http2.Http2CodecUtil.streamableBytes;
import static io.netty.handler.codec.http2.Http2Error.INTERNAL_ERROR;
import static io.netty.handler.codec.http2.Http2Exception.connectionError;
import static java.lang.Integer.MAX_VALUE;
import static java.lang.Math.max;
import static java.lang.Math.min;
/**
@ -43,32 +58,97 @@ import static java.lang.Math.min;
*/
@UnstableApi
public final class WeightedFairQueueByteDistributor implements StreamByteDistributor {
/**
* The initial size of the children map is chosen to be conservative on initial memory allocations under
* the assumption that most streams will have a small number of children. This choice may be
* sub-optimal if when children are present there are many children (i.e. a web page which has many
* dependencies to load).
*
* Visible only for testing!
*/
static final int INITIAL_CHILDREN_MAP_SIZE =
max(1, SystemPropertyUtil.getInt("io.netty.http2.childrenMapSize", 2));
/**
* FireFox currently uses 5 streams to establish QoS classes.
*/
private static final int DEFAULT_MAX_STATE_ONLY_SIZE = 5;
private final Http2Connection.PropertyKey stateKey;
/**
* If there is no Http2Stream object, but we still persist priority information then this is where the state will
* reside.
*/
private final IntObjectMap<State> stateOnlyMap;
/**
* This queue will hold streams that are not active and provides the capability to retain priority for streams which
* have no {@link Http2Stream} object. See {@link StateOnlyComparator} for the priority comparator.
*/
private final PriorityQueue<State> stateOnlyRemovalQueue;
private final Http2Connection connection;
private final State connectionState;
/**
* The minimum number of bytes that we will attempt to allocate to a stream. This is to
* help improve goodput on a per-stream basis.
*/
private int allocationQuantum = 1024;
private int allocationQuantum = DEFAULT_MIN_ALLOCATION_CHUNK;
private final int maxStateOnlySize;
public WeightedFairQueueByteDistributor(Http2Connection connection) {
this(connection, DEFAULT_MAX_STATE_ONLY_SIZE);
}
public WeightedFairQueueByteDistributor(Http2Connection connection, int maxStateOnlySize) {
if (maxStateOnlySize < 0) {
throw new IllegalArgumentException("maxStateOnlySize: " + maxStateOnlySize + " (expected: >0)");
} else if (maxStateOnlySize == 0) {
stateOnlyMap = IntCollections.emptyMap();
stateOnlyRemovalQueue = EmptyPriorityQueue.instance();
} else {
stateOnlyMap = new IntObjectHashMap<State>(maxStateOnlySize);
// +2 because we may exceed the limit by 2 if a new dependency has no associated Http2Stream object. We need
// to create the State objects to put them into the dependency tree, which then impacts priority.
stateOnlyRemovalQueue = new DefaultPriorityQueue<State>(StateOnlyComparator.INSTANCE, maxStateOnlySize + 2);
}
this.maxStateOnlySize = maxStateOnlySize;
this.connection = connection;
stateKey = connection.newKey();
Http2Stream connectionStream = connection.connectionStream();
final Http2Stream connectionStream = connection.connectionStream();
connectionStream.setProperty(stateKey, connectionState = new State(connectionStream, 16));
// Register for notification of new streams.
connection.addListener(new Http2ConnectionAdapter() {
@Override
public void onStreamAdded(Http2Stream stream) {
stream.setProperty(stateKey, new State(stream));
State state = stateOnlyMap.remove(stream.id());
if (state == null) {
state = new State(stream);
// Only the stream which was just added will change parents. So we only need an array of size 1.
List<ParentChangedEvent> events = new ArrayList<ParentChangedEvent>(1);
connectionState.takeChild(state, false, events);
notifyParentChanged(events);
} else {
stateOnlyRemovalQueue.removeTyped(state);
state.stream = stream;
}
switch (stream.state()) {
case RESERVED_REMOTE:
case RESERVED_LOCAL:
state.setStreamReservedOrActivated();
// wasStreamReservedOrActivated is part of the comparator for stateOnlyRemovalQueue there is no
// need to reprioritize here because it will not be in stateOnlyRemovalQueue.
break;
default:
break;
}
stream.setProperty(stateKey, state);
}
@Override
public void onWeightChanged(Http2Stream stream, short oldWeight) {
Http2Stream parent;
if (state(stream).activeCountForTree != 0 && (parent = stream.parent()) != null) {
state(parent).totalQueuedWeights += stream.weight() - oldWeight;
}
public void onStreamActive(Http2Stream stream) {
state(stream).setStreamReservedOrActivated();
// wasStreamReservedOrActivated is part of the comparator for stateOnlyRemovalQueue there is no need to
// reprioritize here because it will not be in stateOnlyRemovalQueue.
}
@Override
@ -77,29 +157,35 @@ public final class WeightedFairQueueByteDistributor implements StreamByteDistrib
}
@Override
public void onPriorityTreeParentChanged(Http2Stream stream, Http2Stream oldParent) {
Http2Stream parent = stream.parent();
if (parent != null) {
State state = state(stream);
if (state.activeCountForTree != 0) {
State pState = state(parent);
pState.offerAndInitializePseudoTime(state);
pState.activeCountChangeForTree(state.activeCountForTree);
}
}
}
public void onStreamRemoved(Http2Stream stream) {
// The stream has been removed from the connection. We can no longer rely on the stream's property
// storage to track the State. If we have room, and the precedence of the stream is sufficient, we
// should retain the State in the stateOnlyMap.
State state = state(stream);
@Override
public void onPriorityTreeParentChanging(Http2Stream stream, Http2Stream newParent) {
Http2Stream parent = stream.parent();
if (parent != null) {
State state = state(stream);
if (state.activeCountForTree != 0) {
State pState = state(parent);
pState.remove(state);
pState.activeCountChangeForTree(-state.activeCountForTree);
}
// Typically the stream is set to null when the stream is closed because it is no longer needed to write
// data. However if the stream was not activated it may not be closed (reserved streams) so we ensure
// the stream reference is set to null to avoid retaining a reference longer than necessary.
state.stream = null;
if (WeightedFairQueueByteDistributor.this.maxStateOnlySize == 0) {
state.parent.removeChild(state);
return;
}
if (stateOnlyRemovalQueue.size() == WeightedFairQueueByteDistributor.this.maxStateOnlySize) {
State stateToRemove = stateOnlyRemovalQueue.peek();
if (StateOnlyComparator.INSTANCE.compare(stateToRemove, state) >= 0) {
// The "lowest priority" stream is a "higher priority" than the stream being removed, so we
// just discard the state.
state.parent.removeChild(state);
return;
}
stateOnlyRemovalQueue.poll();
stateToRemove.parent.removeChild(stateToRemove);
stateOnlyMap.remove(stateToRemove.streamId);
}
stateOnlyRemovalQueue.add(state);
stateOnlyMap.put(state.streamId, state);
}
});
}
@ -110,6 +196,71 @@ public final class WeightedFairQueueByteDistributor implements StreamByteDistrib
state.hasFrame() && state.windowSize() >= 0);
}
@Override
public void updateDependencyTree(int childStreamId, int parentStreamId, short weight, boolean exclusive) {
if (weight < MIN_WEIGHT || weight > MAX_WEIGHT) {
throw new IllegalArgumentException(String.format(
"Invalid weight: %d. Must be between %d and %d (inclusive).", weight, MIN_WEIGHT, MAX_WEIGHT));
}
if (childStreamId == parentStreamId) {
throw new IllegalArgumentException("A stream cannot depend on itself");
}
State state = state(childStreamId);
if (state == null) {
// If there is no State object that means there is no Http2Stream object and we would have to keep the
// State object in the stateOnlyMap and stateOnlyRemovalQueue. However if maxStateOnlySize is 0 this means
// stateOnlyMap and stateOnlyRemovalQueue are empty collections and cannot be modified so we drop the State.
if (maxStateOnlySize == 0) {
return;
}
state = new State(childStreamId);
stateOnlyRemovalQueue.add(state);
stateOnlyMap.put(childStreamId, state);
}
State newParent = state(parentStreamId);
if (newParent == null) {
// If there is no State object that means there is no Http2Stream object and we would have to keep the
// State object in the stateOnlyMap and stateOnlyRemovalQueue. However if maxStateOnlySize is 0 this means
// stateOnlyMap and stateOnlyRemovalQueue are empty collections and cannot be modified so we drop the State.
if (maxStateOnlySize == 0) {
return;
}
newParent = new State(parentStreamId);
stateOnlyRemovalQueue.add(newParent);
stateOnlyMap.put(parentStreamId, newParent);
}
// if activeCountForTree == 0 then it will not be in its parent's pseudoTimeQueue and thus should not be counted
// toward parent.totalQueuedWeights.
if (state.activeCountForTree != 0 && state.parent != null) {
state.parent.totalQueuedWeights += weight - state.weight;
}
state.weight = weight;
if (newParent != state.parent || (exclusive && newParent.children.size() != 1)) {
final List<ParentChangedEvent> events;
if (newParent.isDescendantOf(state)) {
events = new ArrayList<ParentChangedEvent>(2 + (exclusive ? newParent.children.size() : 0));
state.parent.takeChild(newParent, false, events);
} else {
events = new ArrayList<ParentChangedEvent>(1 + (exclusive ? newParent.children.size() : 0));
}
newParent.takeChild(state, exclusive, events);
notifyParentChanged(events);
}
// The location in the dependency tree impacts the priority in the stateOnlyRemovalQueue map. If we created new
// State objects we must check if we exceeded the limit after we insert into the dependency tree to ensure the
// stateOnlyRemovalQueue has been updated.
while (stateOnlyRemovalQueue.size() > maxStateOnlySize) {
State stateToRemove = stateOnlyRemovalQueue.poll();
stateToRemove.parent.removeChild(stateToRemove);
stateOnlyMap.remove(stateToRemove.streamId);
}
}
@Override
public boolean distribute(int maxBytes, Writer writer) throws Http2Exception {
// As long as there is some active frame we should write at least 1 time.
@ -171,17 +322,16 @@ public final class WeightedFairQueueByteDistributor implements StreamByteDistrib
*/
private int distributeToChildren(int maxBytes, Writer writer, State state) throws Http2Exception {
long oldTotalQueuedWeights = state.totalQueuedWeights;
State childState = state.poll();
State nextChildState = state.peek();
State childState = state.pollPseudoTimeQueue();
State nextChildState = state.peekPseudoTimeQueue();
childState.setDistributing();
try {
assert nextChildState == null || nextChildState.pseudoTimeToWrite >= childState.pseudoTimeToWrite :
"nextChildState[" + nextChildState.stream.id() + "].pseudoTime(" + nextChildState.pseudoTimeToWrite +
") < " + " childState[" + childState.stream.id() + "].pseudoTime(" + childState.pseudoTimeToWrite + ")";
"nextChildState[" + nextChildState.streamId + "].pseudoTime(" + nextChildState.pseudoTimeToWrite +
") < " + " childState[" + childState.streamId + "].pseudoTime(" + childState.pseudoTimeToWrite + ")";
int nsent = distribute(nextChildState == null ? maxBytes :
min(maxBytes, (int) min((nextChildState.pseudoTimeToWrite - childState.pseudoTimeToWrite) *
childState.stream.weight() / oldTotalQueuedWeights + allocationQuantum,
Integer.MAX_VALUE)
childState.weight / oldTotalQueuedWeights + allocationQuantum, MAX_VALUE)
),
writer,
childState);
@ -192,9 +342,9 @@ public final class WeightedFairQueueByteDistributor implements StreamByteDistrib
childState.unsetDistributing();
// Do in finally to ensure the internal flags is not corrupted if an exception is thrown.
// The offer operation is delayed until we unroll up the recursive stack, so we don't have to remove from
// the priority queue due to a write operation.
// the priority pseudoTimeQueue due to a write operation.
if (childState.activeCountForTree != 0) {
state.offer(childState);
state.offerPseudoTimeQueue(childState);
}
}
}
@ -203,6 +353,11 @@ public final class WeightedFairQueueByteDistributor implements StreamByteDistrib
return stream.getProperty(stateKey);
}
private State state(int streamId) {
Http2Stream stream = connection.stream(streamId);
return stream != null ? state(stream) : stateOnlyMap.get(streamId);
}
/**
* For testing only!
*/
@ -210,20 +365,110 @@ public final class WeightedFairQueueByteDistributor implements StreamByteDistrib
return state(stream).streamableBytes;
}
/**
* For testing only!
*/
boolean isChild(int childId, int parentId, short weight) {
State parent = state(parentId);
State child;
return parent.children.containsKey(childId) &&
(child = state(childId)).parent == parent && child.weight == weight;
}
/**
* For testing only!
*/
int numChildren(int streamId) {
State state = state(streamId);
return state == null ? 0 : state.children.size();
}
/**
* Notify all listeners of the priority tree change events (in ascending order)
* @param events The events (top down order) which have changed
*/
void notifyParentChanged(List<ParentChangedEvent> events) {
for (int i = 0; i < events.size(); ++i) {
ParentChangedEvent event = events.get(i);
stateOnlyRemovalQueue.priorityChanged(event.state);
if (event.state.parent != null && event.state.activeCountForTree != 0) {
event.state.parent.offerAndInitializePseudoTime(event.state);
event.state.parent.activeCountChangeForTree(event.state.activeCountForTree);
}
}
}
/**
* A comparator for {@link State} which has no associated {@link Http2Stream} object. The general precedence is:
* <ul>
* <li>Was a stream activated or reserved (streams only used for priority are higher priority)</li>
* <li>Depth in the priority tree (closer to root is higher priority></li>
* <li>Stream ID (higher stream ID is higher priority - used for tie breaker)</li>
* </ul>
*/
private static final class StateOnlyComparator implements Comparator<State> {
static final StateOnlyComparator INSTANCE = new StateOnlyComparator();
private StateOnlyComparator() {
}
@Override
public int compare(State o1, State o2) {
// "priority only streams" (which have not been activated) are higher priority than streams used for data.
boolean o1Actived = o1.wasStreamReservedOrActivated();
if (o1Actived != o2.wasStreamReservedOrActivated()) {
return o1Actived ? -1 : 1;
}
// Numerically greater depth is higher priority.
int x = o2.dependencyTreeDepth - o1.dependencyTreeDepth;
// I also considered tracking the number of streams which are "activated" (eligible transfer data) at each
// subtree. This would require a traversal from each node to the root on dependency tree structural changes,
// and then it would require a re-prioritization at each of these nodes (instead of just the nodes where the
// direct parent changed). The costs of this are judged to be relatively high compared to the nominal
// benefit it provides to the heuristic. Instead folks should just increase maxStateOnlySize.
// Last resort is to give larger stream ids more priority.
return x != 0 ? x : o1.streamId - o2.streamId;
}
}
private static final class StatePseudoTimeComparator implements Comparator<State> {
static final StatePseudoTimeComparator INSTANCE = new StatePseudoTimeComparator();
private StatePseudoTimeComparator() {
}
@Override
public int compare(State o1, State o2) {
return MathUtil.compare(o1.pseudoTimeToWrite, o2.pseudoTimeToWrite);
}
}
/**
* The remote flow control state for a single stream.
*/
private final class State implements PriorityQueueNode<State> {
private static final int STATE_IS_ACTIVE = 0x1;
private static final int STATE_IS_DISTRIBUTING = 0x2;
final Http2Stream stream;
private final Queue<State> queue;
private final class State implements PriorityQueueNode {
private static final byte STATE_IS_ACTIVE = 0x1;
private static final byte STATE_IS_DISTRIBUTING = 0x2;
private static final byte STATE_STREAM_ACTIVATED = 0x4;
/**
* Maybe {@code null} if the stream if the stream is not active.
*/
Http2Stream stream;
State parent;
IntObjectMap<State> children = IntCollections.emptyMap();
private final PriorityQueue<State> pseudoTimeQueue;
final int streamId;
int streamableBytes;
int dependencyTreeDepth;
/**
* Count of nodes rooted at this sub tree with {@link #isActive()} equal to {@code true}.
*/
int activeCountForTree;
private int priorityQueueIndex = INDEX_NOT_IN_QUEUE;
private int pseudoTimeQueueIndex = INDEX_NOT_IN_QUEUE;
private int stateOnlyQueueIndex = INDEX_NOT_IN_QUEUE;
/**
* An estimate of when this node should be given the opportunity to write data.
*/
@ -234,17 +479,137 @@ public final class WeightedFairQueueByteDistributor implements StreamByteDistrib
long pseudoTime;
long totalQueuedWeights;
private byte flags;
short weight = DEFAULT_PRIORITY_WEIGHT;
State(int streamId) {
this(streamId, null, 0);
}
State(Http2Stream stream) {
this(stream, 0);
}
State(Http2Stream stream, int initialSize) {
this(stream.id(), stream, initialSize);
}
State(int streamId, Http2Stream stream, int initialSize) {
this.stream = stream;
queue = new PriorityQueue<State>(initialSize);
this.streamId = streamId;
pseudoTimeQueue = new DefaultPriorityQueue<State>(StatePseudoTimeComparator.INSTANCE, initialSize);
}
boolean isDescendantOf(State state) {
State next = parent;
while (next != null) {
if (next == state) {
return true;
}
next = next.parent;
}
return false;
}
void takeChild(State child, boolean exclusive, List<ParentChangedEvent> events) {
takeChild(null, child, exclusive, events);
}
/**
* Adds a child to this priority. If exclusive is set, any children of this node are moved to being dependent on
* the child.
*/
void takeChild(Iterator<IntObjectMap.PrimitiveEntry<State>> childItr, State child, boolean exclusive,
List<ParentChangedEvent> events) {
State oldParent = child.parent;
if (oldParent != this) {
events.add(new ParentChangedEvent(child, oldParent));
child.setParent(this);
// If the childItr is not null we are iterating over the oldParent.children collection and should
// use the iterator to remove from the collection to avoid concurrent modification. Otherwise it is
// assumed we are not iterating over this collection and it is safe to call remove directly.
if (childItr != null) {
childItr.remove();
} else if (oldParent != null) {
oldParent.children.remove(child.streamId);
}
// Lazily initialize the children to save object allocations.
initChildrenIfEmpty();
final State oldChild = children.put(child.streamId, child);
assert oldChild == null : "A stream with the same stream ID was already in the child map.";
}
if (exclusive && !children.isEmpty()) {
// If it was requested that this child be the exclusive dependency of this node,
// move any previous children to the child node, becoming grand children of this node.
Iterator<IntObjectMap.PrimitiveEntry<State>> itr = removeAllChildrenExcept(child).entries().iterator();
while (itr.hasNext()) {
child.takeChild(itr, itr.next().value(), false, events);
}
}
}
/**
* Removes the child priority and moves any of its dependencies to being direct dependencies on this node.
*/
void removeChild(State child) {
if (children.remove(child.streamId) != null) {
List<ParentChangedEvent> events = new ArrayList<ParentChangedEvent>(1 + child.children.size());
events.add(new ParentChangedEvent(child, child.parent));
child.setParent(null);
// Move up any grand children to be directly dependent on this node.
Iterator<IntObjectMap.PrimitiveEntry<State>> itr = child.children.entries().iterator();
while (itr.hasNext()) {
takeChild(itr, itr.next().value(), false, events);
}
notifyParentChanged(events);
}
}
/**
* Remove all children with the exception of {@code streamToRetain}.
* This method is intended to be used to support an exclusive priority dependency operation.
* @return The map of children prior to this operation, excluding {@code streamToRetain} if present.
*/
private IntObjectMap<State> removeAllChildrenExcept(State stateToRetain) {
stateToRetain = children.remove(stateToRetain.streamId);
IntObjectMap<State> prevChildren = children;
// This map should be re-initialized in anticipation for the 1 exclusive child which will be added.
// It will either be added directly in this method, or after this method is called...but it will be added.
initChildren();
if (stateToRetain != null) {
children.put(stateToRetain.streamId, stateToRetain);
}
return prevChildren;
}
private void setParent(State newParent) {
// if activeCountForTree == 0 then it will not be in its parent's pseudoTimeQueue.
if (activeCountForTree != 0 && parent != null) {
parent.removePseudoTimeQueue(this);
parent.activeCountChangeForTree(-activeCountForTree);
}
parent = newParent;
// Use MAX_VALUE if no parent because lower depth is considered higher priority by StateOnlyComparator.
dependencyTreeDepth = newParent == null ? MAX_VALUE : newParent.dependencyTreeDepth + 1;
}
private void initChildrenIfEmpty() {
if (children == IntCollections.<State>emptyMap()) {
initChildren();
}
}
private void initChildren() {
children = new IntObjectHashMap<State>(INITIAL_CHILDREN_MAP_SIZE);
}
void write(int numBytes, Writer writer) throws Http2Exception {
assert stream != null;
try {
writer.write(stream, numBytes);
} catch (Throwable t) {
@ -255,26 +620,26 @@ public final class WeightedFairQueueByteDistributor implements StreamByteDistrib
void activeCountChangeForTree(int increment) {
assert activeCountForTree + increment >= 0;
activeCountForTree += increment;
if (!stream.isRoot()) {
State pState = state(stream.parent());
if (parent != null) {
assert activeCountForTree != increment ||
priorityQueueIndex == INDEX_NOT_IN_QUEUE ||
pState.queue.contains(this) :
"State[" + stream.id() + "].activeCountForTree changed from 0 to " + increment + " is in a queue" +
", but not in parent[ " + pState.stream.id() + "]'s queue";
pseudoTimeQueueIndex == INDEX_NOT_IN_QUEUE ||
parent.pseudoTimeQueue.containsTyped(this) :
"State[" + streamId + "].activeCountForTree changed from 0 to " + increment + " is in a " +
"pseudoTimeQueue, but not in parent[ " + parent.streamId + "]'s pseudoTimeQueue";
if (activeCountForTree == 0) {
pState.remove(this);
parent.removePseudoTimeQueue(this);
} else if (activeCountForTree == increment && !isDistributing()) {
// If frame count was 0 but is now not, and this node is not already in a queue (assumed to be
// pState's queue) then enqueue it. If this State object is being processed the pseudoTime for this
// node should not be adjusted, and the node will be added back to the queue/tree structure after it
// is done being processed. This may happen if the activeCountForTree == 0 (a node which can't
// stream anything and is blocked) is at/near root of the tree, and is poped off the queue during
// processing, and then put back on the queue because a child changes position in the priority tree
// (or is closed because it is not blocked and finished writing all data).
pState.offerAndInitializePseudoTime(this);
// If frame count was 0 but is now not, and this node is not already in a pseudoTimeQueue (assumed
// to be pState's pseudoTimeQueue) then enqueue it. If this State object is being processed the
// pseudoTime for this node should not be adjusted, and the node will be added back to the
// pseudoTimeQueue/tree structure after it is done being processed. This may happen if the
// activeCountForTree == 0 (a node which can't stream anything and is blocked) is at/near root of
// the tree, and is popped off the pseudoTimeQueue during processing, and then put back on the
// pseudoTimeQueue because a child changes position in the priority tree (or is closed because it is
// not blocked and finished writing all data).
parent.offerAndInitializePseudoTime(this);
}
pState.activeCountChangeForTree(increment);
parent.activeCountChangeForTree(increment);
}
}
@ -296,50 +661,58 @@ public final class WeightedFairQueueByteDistributor implements StreamByteDistrib
* Assumes the parents {@link #totalQueuedWeights} includes this node's weight.
*/
void updatePseudoTime(State parentState, int nsent, long totalQueuedWeights) {
assert stream.id() != CONNECTION_STREAM_ID && nsent >= 0;
assert streamId != CONNECTION_STREAM_ID && nsent >= 0;
// If the current pseudoTimeToSend is greater than parentState.pseudoTime then we previously over accounted
// and should use parentState.pseudoTime.
pseudoTimeToWrite = min(pseudoTimeToWrite, parentState.pseudoTime) +
nsent * totalQueuedWeights / stream.weight();
pseudoTimeToWrite = min(pseudoTimeToWrite, parentState.pseudoTime) + nsent * totalQueuedWeights / weight;
}
/**
* The concept of pseudoTime can be influenced by priority tree manipulations or if a stream goes from "active"
* to "non-active". This method accounts for that by initializing the {@link #pseudoTimeToWrite} for
* {@code state} to {@link #pseudoTime} of this node and then calls {@link #offer(State)}.
* {@code state} to {@link #pseudoTime} of this node and then calls {@link #offerPseudoTimeQueue(State)}.
*/
void offerAndInitializePseudoTime(State state) {
state.pseudoTimeToWrite = pseudoTime;
offer(state);
offerPseudoTimeQueue(state);
}
void offer(State state) {
queue.offer(state);
totalQueuedWeights += state.stream.weight();
void offerPseudoTimeQueue(State state) {
pseudoTimeQueue.offer(state);
totalQueuedWeights += state.weight;
}
/**
* Must only be called if the queue is non-empty!
* Must only be called if the pseudoTimeQueue is non-empty!
*/
State poll() {
State state = queue.poll();
// This method is only ever called if the queue is non-empty.
totalQueuedWeights -= state.stream.weight();
State pollPseudoTimeQueue() {
State state = pseudoTimeQueue.poll();
// This method is only ever called if the pseudoTimeQueue is non-empty.
totalQueuedWeights -= state.weight;
return state;
}
void remove(State state) {
if (queue.remove(state)) {
totalQueuedWeights -= state.stream.weight();
void removePseudoTimeQueue(State state) {
if (pseudoTimeQueue.removeTyped(state)) {
totalQueuedWeights -= state.weight;
}
}
State peek() {
return queue.peek();
State peekPseudoTimeQueue() {
return pseudoTimeQueue.peek();
}
void close() {
updateStreamableBytes(0, false);
stream = null;
}
boolean wasStreamReservedOrActivated() {
return (flags & STATE_STREAM_ACTIVATED) != 0;
}
void setStreamReservedOrActivated() {
flags |= STATE_STREAM_ACTIVATED;
}
boolean isActive() {
@ -367,18 +740,17 @@ public final class WeightedFairQueueByteDistributor implements StreamByteDistrib
}
@Override
public int compareTo(State o) {
return MathUtil.compare(pseudoTimeToWrite, o.pseudoTimeToWrite);
public int priorityQueueIndex(DefaultPriorityQueue<?> queue) {
return queue == stateOnlyRemovalQueue ? stateOnlyQueueIndex : pseudoTimeQueueIndex;
}
@Override
public int priorityQueueIndex() {
return priorityQueueIndex;
}
@Override
public void priorityQueueIndex(int i) {
priorityQueueIndex = i;
public void priorityQueueIndex(DefaultPriorityQueue<?> queue, int i) {
if (queue == stateOnlyRemovalQueue) {
stateOnlyQueueIndex = i;
} else {
pseudoTimeQueueIndex = i;
}
}
@Override
@ -390,17 +762,19 @@ public final class WeightedFairQueueByteDistributor implements StreamByteDistrib
}
private void toString(StringBuilder sb) {
sb.append("{stream ").append(stream.id())
sb.append("{streamId ").append(streamId)
.append(" streamableBytes ").append(streamableBytes)
.append(" activeCountForTree ").append(activeCountForTree)
.append(" priorityQueueIndex ").append(priorityQueueIndex)
.append(" pseudoTimeQueueIndex ").append(pseudoTimeQueueIndex)
.append(" pseudoTimeToWrite ").append(pseudoTimeToWrite)
.append(" pseudoTime ").append(pseudoTime)
.append(" flags ").append(flags)
.append(" queue.size() ").append(queue.size()).append("} [");
.append(" pseudoTimeQueue.size() ").append(pseudoTimeQueue.size())
.append(" stateOnlyQueueIndex ").append(stateOnlyQueueIndex)
.append(" parent ").append(parent).append("} [");
if (!queue.isEmpty()) {
for (State s : queue) {
if (!pseudoTimeQueue.isEmpty()) {
for (State s : pseudoTimeQueue) {
s.toString(sb);
sb.append(", ");
}
@ -410,4 +784,22 @@ public final class WeightedFairQueueByteDistributor implements StreamByteDistrib
sb.append(']');
}
}
/**
* Allows a correlation to be made between a stream and its old parent before a parent change occurs.
*/
private static final class ParentChangedEvent {
final State state;
final State oldParent;
/**
* Create a new instance.
* @param state The state who has had a parent change.
* @param oldParent The previous parent.
*/
ParentChangedEvent(State state, State oldParent) {
this.state = state;
this.oldParent = oldParent;
}
}
}

View File

@ -0,0 +1,82 @@
/*
* Copyright 2017 The Netty Project
*
* The Netty Project 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 io.netty.handler.codec.http2;
import org.mockito.Mock;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
abstract class AbstractWeightedFairQueueByteDistributorDependencyTest {
Http2Connection connection;
WeightedFairQueueByteDistributor distributor;
@Mock
StreamByteDistributor.Writer writer;
Http2Stream stream(int streamId) {
return connection.stream(streamId);
}
Answer<Void> writeAnswer(final boolean closeIfNoFrame) {
return new Answer<Void>() {
@Override
public Void answer(InvocationOnMock in) throws Throwable {
Http2Stream stream = in.getArgumentAt(0, Http2Stream.class);
int numBytes = in.getArgumentAt(1, Integer.class);
int streamableBytes = distributor.streamableBytes0(stream) - numBytes;
boolean hasFrame = streamableBytes > 0;
updateStream(stream.id(), streamableBytes, hasFrame, hasFrame, closeIfNoFrame);
return null;
}
};
}
void updateStream(final int streamId, final int streamableBytes, final boolean hasFrame) {
updateStream(streamId, streamableBytes, hasFrame, hasFrame, false);
}
void updateStream(final int streamId, final int pendingBytes, final boolean hasFrame,
final boolean isWriteAllowed, boolean closeIfNoFrame) {
final Http2Stream stream = stream(streamId);
if (closeIfNoFrame && !hasFrame) {
stream(streamId).close();
}
distributor.updateStreamableBytes(new StreamByteDistributor.StreamState() {
@Override
public Http2Stream stream() {
return stream;
}
@Override
public int pendingBytes() {
return pendingBytes;
}
@Override
public boolean hasFrame() {
return hasFrame;
}
@Override
public int windowSize() {
return isWriteAllowed ? pendingBytes : -1;
}
});
}
void setPriority(int streamId, int parent, int weight, boolean exclusive) throws Http2Exception {
distributor.updateDependencyTree(streamId, parent, (short) weight, exclusive);
}
}

View File

@ -14,6 +14,25 @@
*/
package io.netty.handler.codec.http2;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.UnpooledByteBufAllocator;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelPromise;
import io.netty.channel.DefaultChannelPromise;
import junit.framework.AssertionFailedError;
import org.junit.Before;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.mockito.Mock;
import org.mockito.MockitoAnnotations;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.util.Collections;
import java.util.concurrent.atomic.AtomicInteger;
import static io.netty.buffer.Unpooled.EMPTY_BUFFER;
import static io.netty.buffer.Unpooled.wrappedBuffer;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT;
@ -41,26 +60,6 @@ import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.when;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.UnpooledByteBufAllocator;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelPromise;
import io.netty.channel.DefaultChannelPromise;
import io.netty.handler.codec.http2.Http2Exception.ClosedStreamCreationException;
import junit.framework.AssertionFailedError;
import org.junit.Before;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.mockito.Mock;
import org.mockito.MockitoAnnotations;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.util.Collections;
import java.util.concurrent.atomic.AtomicInteger;
/**
* Tests for {@link DefaultHttp2ConnectionDecoder}.
*/
@ -141,9 +140,7 @@ public class DefaultHttp2ConnectionDecoderTest {
when(local.flowController()).thenReturn(localFlow);
when(encoder.flowController()).thenReturn(remoteFlow);
when(connection.remote()).thenReturn(remote);
when(local.createIdleStream(eq(STREAM_ID))).thenReturn(stream);
when(local.reservePushStream(eq(PUSH_STREAM_ID), eq(stream))).thenReturn(pushStream);
when(remote.createIdleStream(eq(STREAM_ID))).thenReturn(stream);
when(remote.reservePushStream(eq(PUSH_STREAM_ID), eq(stream))).thenReturn(pushStream);
when(ctx.alloc()).thenReturn(UnpooledByteBufAllocator.DEFAULT);
when(ctx.channel()).thenReturn(channel);
@ -399,13 +396,11 @@ public class DefaultHttp2ConnectionDecoderTest {
public void headersReadForStreamThatAlreadySentResetShouldBeIgnored() throws Exception {
when(stream.isResetSent()).thenReturn(true);
decode().onHeadersRead(ctx, STREAM_ID, EmptyHttp2Headers.INSTANCE, 0, false);
verify(remote, never()).createIdleStream(anyInt());
verify(remote, never()).createStream(anyInt(), anyBoolean());
verify(stream, never()).open(anyBoolean());
// Verify that the event was absorbed and not propagated to the oberver.
verify(listener, never()).onHeadersRead(eq(ctx), anyInt(), any(Http2Headers.class), anyInt(), anyBoolean());
verify(remote, never()).createIdleStream(anyInt());
verify(remote, never()).createStream(anyInt(), anyBoolean());
verify(stream, never()).open(anyBoolean());
}
@ -470,44 +465,10 @@ public class DefaultHttp2ConnectionDecoderTest {
weight, true, 0, true);
verify(listener).onHeadersRead(eq(ctx), eq(STREAM_ID), eq(EmptyHttp2Headers.INSTANCE), eq(STREAM_DEPENDENCY_ID),
eq(weight), eq(true), eq(0), eq(true));
verify(stream).setPriority(eq(STREAM_DEPENDENCY_ID), eq(weight), eq(true));
verify(remoteFlow).updateDependencyTree(eq(STREAM_ID), eq(STREAM_DEPENDENCY_ID), eq(weight), eq(true));
verify(lifecycleManager).closeStreamRemote(eq(stream), any(ChannelFuture.class));
}
@Test
public void headersDependencyPreviouslyCreatedStreamShouldSucceed() throws Exception {
final short weight = 1;
doAnswer(new Answer<Http2Stream>() {
@Override
public Http2Stream answer(InvocationOnMock in) throws Throwable {
throw new ClosedStreamCreationException(Http2Error.INTERNAL_ERROR);
}
}).when(stream).setPriority(eq(STREAM_DEPENDENCY_ID), eq(weight), eq(true));
decode().onHeadersRead(ctx, STREAM_ID, EmptyHttp2Headers.INSTANCE, STREAM_DEPENDENCY_ID,
weight, true, 0, true);
verify(listener).onHeadersRead(eq(ctx), eq(STREAM_ID), eq(EmptyHttp2Headers.INSTANCE), eq(STREAM_DEPENDENCY_ID),
eq(weight), eq(true), eq(0), eq(true));
verify(stream).setPriority(eq(STREAM_DEPENDENCY_ID), eq(weight), eq(true));
verify(lifecycleManager).closeStreamRemote(eq(stream), any(ChannelFuture.class));
}
@Test(expected = RuntimeException.class)
public void headersDependencyInvalidStreamShouldFail() throws Exception {
final short weight = 1;
doAnswer(new Answer<Http2Stream>() {
@Override
public Http2Stream answer(InvocationOnMock in) throws Throwable {
throw new RuntimeException("Fake Exception");
}
}).when(stream).setPriority(eq(STREAM_DEPENDENCY_ID), eq(weight), eq(true));
decode().onHeadersRead(ctx, STREAM_ID, EmptyHttp2Headers.INSTANCE, STREAM_DEPENDENCY_ID,
weight, true, 0, true);
verify(listener, never()).onHeadersRead(any(ChannelHandlerContext.class), anyInt(), any(Http2Headers.class),
anyInt(), anyShort(), anyBoolean(), anyInt(), anyBoolean());
verify(stream).setPriority(eq(STREAM_DEPENDENCY_ID), eq(weight), eq(true));
verify(lifecycleManager, never()).closeStreamRemote(eq(stream), any(ChannelFuture.class));
}
@Test
public void pushPromiseReadAfterGoAwaySentShouldBeIgnored() throws Exception {
mockGoAwaySent();
@ -538,54 +499,33 @@ public class DefaultHttp2ConnectionDecoderTest {
eq(EmptyHttp2Headers.INSTANCE), eq(0));
}
@Test
public void priorityReadAfterGoAwaySentShouldBeIgnored() throws Exception {
mockGoAwaySent();
decode().onPriorityRead(ctx, STREAM_ID, 0, (short) 255, true);
verify(stream, never()).setPriority(anyInt(), anyShort(), anyBoolean());
verify(listener, never()).onPriorityRead(eq(ctx), anyInt(), anyInt(), anyShort(), anyBoolean());
}
@Test
public void priorityReadAfterGoAwaySentShouldAllowFramesForStreamCreatedByLocalEndpoint() throws Exception {
mockGoAwaySentShouldAllowFramesForStreamCreatedByLocalEndpoint();
decode().onPriorityRead(ctx, STREAM_ID, 0, (short) 255, true);
verify(stream).setPriority(anyInt(), anyShort(), anyBoolean());
verify(remoteFlow).updateDependencyTree(eq(STREAM_ID), eq(0), eq((short) 255), eq(true));
verify(listener).onPriorityRead(eq(ctx), anyInt(), anyInt(), anyShort(), anyBoolean());
}
@Test
public void priorityReadForUnknownStreamShouldBeIgnored() throws Exception {
public void priorityReadForUnknownStreamShouldNotBeIgnored() throws Exception {
when(connection.stream(STREAM_ID)).thenReturn(null);
decode().onPriorityRead(ctx, STREAM_ID, 0, (short) 255, true);
verify(stream, never()).setPriority(anyInt(), anyShort(), anyBoolean());
verify(listener, never()).onPriorityRead(eq(ctx), anyInt(), anyInt(), anyShort(), anyBoolean());
verify(remoteFlow).updateDependencyTree(eq(STREAM_ID), eq(0), eq((short) 255), eq(true));
verify(listener).onPriorityRead(eq(ctx), eq(STREAM_ID), eq(0), eq((short) 255), eq(true));
}
@Test
public void priorityReadShouldCreateNewStream() throws Exception {
public void priorityReadShouldNotCreateNewStream() throws Exception {
when(connection.streamMayHaveExisted(STREAM_ID)).thenReturn(false);
when(connection.stream(STREAM_ID)).thenReturn(null);
decode().onPriorityRead(ctx, STREAM_ID, STREAM_DEPENDENCY_ID, (short) 255, true);
verify(stream).setPriority(eq(STREAM_DEPENDENCY_ID), eq((short) 255), eq(true));
verify(remoteFlow).updateDependencyTree(eq(STREAM_ID), eq(STREAM_DEPENDENCY_ID), eq((short) 255), eq(true));
verify(listener).onPriorityRead(eq(ctx), eq(STREAM_ID), eq(STREAM_DEPENDENCY_ID), eq((short) 255), eq(true));
verify(remote).createIdleStream(STREAM_ID);
verify(remote, never()).createStream(eq(STREAM_ID), anyBoolean());
verify(stream, never()).open(anyBoolean());
}
@Test
public void priorityReadOnPreviouslyParentExistingStreamShouldSucceed() throws Exception {
doAnswer(new Answer<Http2Stream>() {
@Override
public Http2Stream answer(InvocationOnMock in) throws Throwable {
throw new ClosedStreamCreationException(Http2Error.INTERNAL_ERROR);
}
}).when(stream).setPriority(eq(STREAM_DEPENDENCY_ID), eq((short) 255), eq(true));
decode().onPriorityRead(ctx, STREAM_ID, STREAM_DEPENDENCY_ID, (short) 255, true);
verify(stream).setPriority(eq(STREAM_DEPENDENCY_ID), eq((short) 255), eq(true));
verify(listener).onPriorityRead(eq(ctx), eq(STREAM_ID), eq(STREAM_DEPENDENCY_ID), eq((short) 255), eq(true));
}
@Test
public void windowUpdateReadAfterGoAwaySentShouldBeIgnored() throws Exception {
mockGoAwaySent();

View File

@ -14,36 +14,6 @@
*/
package io.netty.handler.codec.http2;
import static io.netty.buffer.Unpooled.EMPTY_BUFFER;
import static io.netty.buffer.Unpooled.wrappedBuffer;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT;
import static io.netty.handler.codec.http2.Http2CodecUtil.emptyPingBuf;
import static io.netty.handler.codec.http2.Http2Error.PROTOCOL_ERROR;
import static io.netty.handler.codec.http2.Http2Stream.State.HALF_CLOSED_REMOTE;
import static io.netty.handler.codec.http2.Http2Stream.State.IDLE;
import static io.netty.handler.codec.http2.Http2Stream.State.RESERVED_LOCAL;
import static io.netty.handler.codec.http2.Http2TestUtil.newVoidPromise;
import static io.netty.util.CharsetUtil.UTF_8;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyBoolean;
import static org.mockito.Matchers.anyInt;
import static org.mockito.Matchers.anyLong;
import static org.mockito.Matchers.anyShort;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.when;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.PooledByteBufAllocator;
import io.netty.buffer.Unpooled;
@ -68,6 +38,36 @@ import org.mockito.stubbing.Answer;
import java.util.ArrayList;
import java.util.List;
import static io.netty.buffer.Unpooled.EMPTY_BUFFER;
import static io.netty.buffer.Unpooled.wrappedBuffer;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT;
import static io.netty.handler.codec.http2.Http2CodecUtil.emptyPingBuf;
import static io.netty.handler.codec.http2.Http2Error.PROTOCOL_ERROR;
import static io.netty.handler.codec.http2.Http2Stream.State.HALF_CLOSED_REMOTE;
import static io.netty.handler.codec.http2.Http2Stream.State.RESERVED_LOCAL;
import static io.netty.handler.codec.http2.Http2TestUtil.newVoidPromise;
import static io.netty.util.CharsetUtil.UTF_8;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyBoolean;
import static org.mockito.Matchers.anyInt;
import static org.mockito.Matchers.anyLong;
import static org.mockito.Matchers.anyShort;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.when;
/**
* Tests for {@link DefaultHttp2ConnectionEncoder}
*/
@ -87,9 +87,6 @@ public class DefaultHttp2ConnectionEncoderTest {
@Mock
private ChannelPipeline pipeline;
@Mock
private Http2FrameListener listener;
@Mock
private Http2FrameWriter writer;
@ -380,10 +377,9 @@ public class DefaultHttp2ConnectionEncoderTest {
short weight = 255;
encoder.writePriority(ctx, STREAM_ID, 0, weight, true, promise);
// Verify that this created an idle stream with the desired weight.
// Verify that this did NOT create a stream object.
Http2Stream stream = stream(STREAM_ID);
assertEquals(IDLE, stream.state());
assertEquals(weight, stream.weight());
assertNull(stream);
verify(writer).writePriority(eq(ctx), eq(STREAM_ID), eq(0), eq((short) 255), eq(true), eq(promise));
}

View File

@ -23,45 +23,33 @@ import io.netty.handler.codec.http2.Http2Stream.State;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.FutureListener;
import io.netty.util.concurrent.Promise;
import io.netty.util.internal.PlatformDependent;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.mockito.ArgumentCaptor;
import org.mockito.Mock;
import org.mockito.MockitoAnnotations;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import static io.netty.handler.codec.http2.DefaultHttp2Connection.INITIAL_CHILDREN_MAP_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT;
import static io.netty.handler.codec.http2.Http2CodecUtil.MIN_WEIGHT;
import static java.lang.Integer.MAX_VALUE;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyInt;
import static org.mockito.Matchers.anyLong;
import static org.mockito.Matchers.anyShort;
import static org.mockito.Matchers.eq;
import static org.mockito.Matchers.isNull;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.reset;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
/**
@ -147,22 +135,6 @@ public class DefaultHttp2ConnectionTest {
testRemoveAllStreams();
}
@Test
public void removeAllStreamsWithNonActiveStreams() throws InterruptedException, Http2Exception {
client.local().createIdleStream(3);
client.remote().createIdleStream(2);
testRemoveAllStreams();
}
@Test
public void removeAllStreamsWithNonActiveAndActiveStreams() throws InterruptedException, Http2Exception {
client.local().createIdleStream(3);
client.remote().createIdleStream(2);
client.local().createStream(5, false);
client.remote().createStream(4, true);
testRemoveAllStreams();
}
@Test
public void removeIndividualStreamsWhileCloseDoesNotNPE() throws InterruptedException, Http2Exception {
final Http2Stream streamA = client.local().createStream(3, false);
@ -183,22 +155,6 @@ public class DefaultHttp2ConnectionTest {
}
}
@Test
public void closingStreamWithChildrenDoesNotCauseConcurrentModification() throws Http2Exception {
// We create enough streams to wrap around the child array. We carefully craft the stream ids so that they hash
// codes overlap with respect to the child collection. If the implementation is not careful this may lead to a
// concurrent modification excpetion while promoting all children to the connection stream.
final Http2Stream streamA = client.local().createStream(1, false);
final int numStreams = INITIAL_CHILDREN_MAP_SIZE - 1;
for (int i = 0, streamId = 3; i < numStreams; ++i, streamId += INITIAL_CHILDREN_MAP_SIZE) {
final Http2Stream stream = client.local().createStream(streamId, false);
stream.setPriority(streamA.id(), Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT, false);
}
assertEquals(INITIAL_CHILDREN_MAP_SIZE, client.numActiveStreams());
streamA.close();
assertEquals(numStreams, client.numActiveStreams());
}
@Test
public void removeAllStreamsWhileIteratingActiveStreams() throws InterruptedException, Http2Exception {
final Endpoint<Http2RemoteFlowController> remote = client.remote();
@ -259,22 +215,6 @@ public class DefaultHttp2ConnectionTest {
assertTrue(latch.await(5, TimeUnit.SECONDS));
}
@Test
public void closeWhileIteratingDoesNotNPE() throws Http2Exception {
final Http2Stream streamA = client.local().createStream(3, false);
final Http2Stream streamB = client.local().createStream(5, false);
final Http2Stream streamC = client.local().createStream(7, false);
streamB.setPriority(streamA.id(), Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT, false);
client.forEachActiveStream(new Http2StreamVisitor() {
@Override
public boolean visit(Http2Stream stream) throws Http2Exception {
streamA.close();
streamB.setPriority(streamC.id(), Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT, false);
return true;
}
});
}
@Test
public void goAwayReceivedShouldCloseStreamsGreaterThanLastStream() throws Exception {
Http2Stream stream1 = client.local().createStream(3, false);
@ -445,38 +385,32 @@ public class DefaultHttp2ConnectionTest {
@Test(expected = Http2Exception.class)
public void createShouldThrowWhenMaxAllowedStreamsOpenExceeded() throws Http2Exception {
server.local().maxStreams(0, 0);
server.local().maxActiveStreams(0);
server.local().createStream(2, true);
}
@Test(expected = Http2Exception.class)
public void createShouldThrowWhenMaxAllowedStreamsIdleExceeded() throws Http2Exception {
server.local().maxStreams(0, 0);
server.local().createIdleStream(2);
public void serverCreatePushShouldFailOnRemoteEndpointWhenMaxAllowedStreamsExceeded() throws Http2Exception {
server = new DefaultHttp2Connection(true, 0);
server.remote().maxActiveStreams(1);
Http2Stream requestStream = server.remote().createStream(3, false);
server.remote().reservePushStream(2, requestStream);
}
@Test(expected = Http2Exception.class)
public void createShouldThrowWhenMaxAllowedStreamsReservedExceeded() throws Http2Exception {
server.local().maxStreams(1, 1);
Http2Stream parent = server.local().createStream(2, false);
server.local().reservePushStream(4, parent);
public void clientCreatePushShouldFailOnRemoteEndpointWhenMaxAllowedStreamsExceeded() throws Http2Exception {
client = new DefaultHttp2Connection(false, 0);
client.remote().maxActiveStreams(1);
Http2Stream requestStream = client.remote().createStream(2, false);
client.remote().reservePushStream(4, requestStream);
}
@Test
public void createIdleShouldSucceedWhenMaxAllowedActiveStreamsExceeded() throws Http2Exception {
server.local().maxStreams(0, MAX_VALUE);
Http2Stream stream = server.local().createIdleStream(2);
// Opening should fail, however.
thrown.expect(Http2Exception.class);
thrown.expectMessage("Maximum active streams violated for this endpoint.");
stream.open(false);
}
@Test(expected = Http2Exception.class)
public void createIdleShouldFailWhenMaxAllowedStreamsExceeded() throws Http2Exception {
server.local().maxStreams(0, 0);
server.local().createIdleStream(2);
public void serverCreatePushShouldSucceedOnLocalEndpointWhenMaxAllowedStreamsExceeded() throws Http2Exception {
server = new DefaultHttp2Connection(true, 0);
server.local().maxActiveStreams(1);
Http2Stream requestStream = server.remote().createStream(3, false);
assertNotNull(server.local().reservePushStream(2, requestStream));
}
@Test(expected = Http2Exception.class)
@ -536,525 +470,6 @@ public class DefaultHttp2ConnectionTest {
client.remote().createStream(MAX_VALUE + 1, false);
}
@Test
public void localStreamCanDependUponIdleStream() throws Http2Exception {
Http2Stream streamA = client.local().createStream(1, false);
streamA.setPriority(3, MIN_WEIGHT, true);
verifyDependUponIdleStream(streamA, client.stream(3), client.local());
}
@Test
public void remoteStreamCanDependUponIdleStream() throws Http2Exception {
Http2Stream streamA = client.remote().createStream(2, false);
streamA.setPriority(4, MIN_WEIGHT, true);
verifyDependUponIdleStream(streamA, client.stream(4), client.remote());
}
@Test
public void prioritizeShouldUseDefaults() throws Exception {
Http2Stream stream = client.local().createStream(1, false);
assertEquals(1, client.connectionStream().numChildren());
assertEquals(stream, child(client.connectionStream(), 1));
assertEquals(DEFAULT_PRIORITY_WEIGHT, stream.weight());
assertEquals(0, stream.parent().id());
assertEquals(0, stream.numChildren());
}
@Test
public void reprioritizeWithNoChangeShouldDoNothing() throws Exception {
Http2Stream stream = client.local().createStream(1, false);
stream.setPriority(0, DEFAULT_PRIORITY_WEIGHT, false);
assertEquals(1, client.connectionStream().numChildren());
assertEquals(stream, child(client.connectionStream(), 1));
assertEquals(DEFAULT_PRIORITY_WEIGHT, stream.weight());
assertEquals(0, stream.parent().id());
assertEquals(0, stream.numChildren());
}
@Test
public void insertExclusiveShouldAddNewLevel() throws Exception {
Http2Stream streamA = client.local().createStream(1, false);
Http2Stream streamB = client.local().createStream(3, false);
Http2Stream streamC = client.local().createStream(5, false);
Http2Stream streamD = client.local().createStream(7, false);
streamB.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamC.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamD.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, true);
assertEquals(4, client.numActiveStreams());
// Level 0
Http2Stream p = client.connectionStream();
assertEquals(1, p.numChildren());
// Level 1
p = child(p, streamA.id());
assertNotNull(p);
assertEquals(0, p.parent().id());
assertEquals(1, p.numChildren());
// Level 2
p = child(p, streamD.id());
assertNotNull(p);
assertEquals(streamA.id(), p.parent().id());
assertEquals(2, p.numChildren());
// Level 3
p = child(p, streamB.id());
assertNotNull(p);
assertEquals(streamD.id(), p.parent().id());
assertEquals(0, p.numChildren());
p = child(p.parent(), streamC.id());
assertNotNull(p);
assertEquals(streamD.id(), p.parent().id());
assertEquals(0, p.numChildren());
}
@Test
public void existingChildMadeExclusiveShouldNotCreateTreeCycle() throws Http2Exception {
Http2Stream streamA = client.local().createStream(1, false);
Http2Stream streamB = client.local().createStream(3, false);
Http2Stream streamC = client.local().createStream(5, false);
Http2Stream streamD = client.local().createStream(7, false);
streamB.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamC.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamD.setPriority(streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
// Stream C is already dependent on Stream A, but now make that an exclusive dependency
streamC.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, true);
assertEquals(4, client.numActiveStreams());
// Level 0
Http2Stream p = client.connectionStream();
assertEquals(1, p.numChildren());
// Level 1
p = child(p, streamA.id());
assertNotNull(p);
assertEquals(0, p.parent().id());
assertEquals(1, p.numChildren());
// Level 2
p = child(p, streamC.id());
assertNotNull(p);
assertEquals(streamA.id(), p.parent().id());
assertEquals(2, p.numChildren());
// Level 3
p = child(p, streamB.id());
assertNotNull(p);
assertEquals(streamC.id(), p.parent().id());
assertEquals(0, p.numChildren());
p = child(p.parent(), streamD.id());
assertNotNull(p);
assertEquals(streamC.id(), p.parent().id());
assertEquals(0, p.numChildren());
}
@Test
public void newExclusiveChildShouldUpdateOldParentCorrectly() throws Http2Exception {
Http2Stream streamA = client.local().createStream(1, false);
Http2Stream streamB = client.local().createStream(3, false);
Http2Stream streamC = client.local().createStream(5, false);
Http2Stream streamD = client.local().createStream(7, false);
Http2Stream streamE = client.local().createStream(9, false);
Http2Stream streamF = client.local().createStream(11, false);
streamB.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamC.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamD.setPriority(streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamF.setPriority(streamE.id(), DEFAULT_PRIORITY_WEIGHT, false);
// F is now going to be exclusively dependent on A, after this we should check that stream E
// prioritizableForTree is not over decremented.
streamF.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, true);
assertEquals(6, client.numActiveStreams());
// Level 0
Http2Stream p = client.connectionStream();
assertEquals(2, p.numChildren());
// Level 1
p = child(p, streamE.id());
assertNotNull(p);
assertEquals(0, p.parent().id());
assertEquals(0, p.numChildren());
p = child(p.parent(), streamA.id());
assertNotNull(p);
assertEquals(0, p.parent().id());
assertEquals(1, p.numChildren());
// Level 2
p = child(p, streamF.id());
assertNotNull(p);
assertEquals(streamA.id(), p.parent().id());
assertEquals(2, p.numChildren());
// Level 3
p = child(p, streamB.id());
assertNotNull(p);
assertEquals(streamF.id(), p.parent().id());
assertEquals(0, p.numChildren());
p = child(p.parent(), streamC.id());
assertNotNull(p);
assertEquals(streamF.id(), p.parent().id());
assertEquals(1, p.numChildren());
// Level 4
p = child(p, streamD.id());
assertNotNull(p);
assertEquals(streamC.id(), p.parent().id());
assertEquals(0, p.numChildren());
}
@Test
public void weightChangeWithNoTreeChangeShouldNotifyListeners() throws Http2Exception {
Http2Stream streamA = client.local().createStream(1, false);
Http2Stream streamB = client.local().createStream(3, false);
Http2Stream streamC = client.local().createStream(5, false);
Http2Stream streamD = client.local().createStream(7, false);
streamB.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamC.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamD.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, true);
assertEquals(4, client.numActiveStreams());
short oldWeight = streamD.weight();
short newWeight = (short) (oldWeight + 1);
reset(clientListener);
streamD.setPriority(streamD.parent().id(), newWeight, false);
verify(clientListener).onWeightChanged(eq(streamD), eq(oldWeight));
assertEquals(streamD.weight(), newWeight);
verify(clientListener, never()).onPriorityTreeParentChanging(any(Http2Stream.class),
any(Http2Stream.class));
verify(clientListener, never()).onPriorityTreeParentChanged(any(Http2Stream.class),
any(Http2Stream.class));
}
@Test
public void sameNodeDependentShouldNotStackOverflowNorChangePrioritizableForTree() throws Http2Exception {
Http2Stream streamA = client.local().createStream(1, false);
Http2Stream streamB = client.local().createStream(3, false);
Http2Stream streamC = client.local().createStream(5, false);
Http2Stream streamD = client.local().createStream(7, false);
streamB.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamC.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamD.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, true);
boolean[] exclusive = new boolean[] {true, false};
short[] weights = new short[] { DEFAULT_PRIORITY_WEIGHT, 100, 200, streamD.weight() };
assertEquals(4, client.numActiveStreams());
Http2Stream connectionStream = client.connectionStream();
// The goal is to call setPriority with the same parent and vary the parameters
// we were at one point adding a circular depends to the tree and then throwing
// a StackOverflow due to infinite recursive operation.
for (int j = 0; j < weights.length; ++j) {
for (int i = 0; i < exclusive.length; ++i) {
streamD.setPriority(streamA.id(), weights[j], exclusive[i]);
}
}
}
@Test
public void multipleCircularDependencyShouldUpdatePrioritizable() throws Http2Exception {
Http2Stream streamA = client.local().createStream(1, false);
Http2Stream streamB = client.local().createStream(3, false);
Http2Stream streamC = client.local().createStream(5, false);
Http2Stream streamD = client.local().createStream(7, false);
streamB.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamC.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamD.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, true);
assertEquals(4, client.numActiveStreams());
Http2Stream connectionStream = client.connectionStream();
// Bring B to the root
streamA.setPriority(streamB.id(), DEFAULT_PRIORITY_WEIGHT, true);
// Move all streams to be children of B
streamC.setPriority(streamB.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamD.setPriority(streamB.id(), DEFAULT_PRIORITY_WEIGHT, false);
// Move A back to the root
streamB.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, true);
// Move all streams to be children of A
streamC.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamD.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
}
@Test
public void removeWithPrioritizableDependentsShouldNotRestructureTree() throws Exception {
Http2Stream streamA = client.local().createStream(1, false);
Http2Stream streamB = client.local().createStream(3, false);
Http2Stream streamC = client.local().createStream(5, false);
Http2Stream streamD = client.local().createStream(7, false);
streamB.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamC.setPriority(streamB.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamD.setPriority(streamB.id(), DEFAULT_PRIORITY_WEIGHT, false);
// Default removal policy will cause it to be removed immediately.
streamB.close();
// Level 0
Http2Stream p = client.connectionStream();
assertEquals(1, p.numChildren());
// Level 1
p = child(p, streamA.id());
assertNotNull(p);
assertEquals(client.connectionStream().id(), p.parent().id());
assertEquals(2, p.numChildren());
// Level 2
p = child(p, streamC.id());
assertNotNull(p);
assertEquals(streamA.id(), p.parent().id());
assertEquals(0, p.numChildren());
p = child(p.parent(), streamD.id());
assertNotNull(p);
assertEquals(streamA.id(), p.parent().id());
assertEquals(0, p.numChildren());
}
@Test
public void closeWithNoPrioritizableDependentsShouldRestructureTree() throws Exception {
Http2Stream streamA = client.local().createStream(1, false);
Http2Stream streamB = client.local().createStream(3, false);
Http2Stream streamC = client.local().createStream(5, false);
Http2Stream streamD = client.local().createStream(7, false);
Http2Stream streamE = client.local().createStream(9, false);
Http2Stream streamF = client.local().createStream(11, false);
streamB.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamC.setPriority(streamB.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamD.setPriority(streamB.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamE.setPriority(streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamF.setPriority(streamD.id(), DEFAULT_PRIORITY_WEIGHT, false);
// Close internal nodes, leave 1 leaf node open, the only remaining stream is the one that is not closed (E).
streamA.close();
streamB.close();
streamC.close();
streamD.close();
streamF.close();
// Level 0
Http2Stream p = client.connectionStream();
assertEquals(1, p.numChildren());
// Level 1
p = child(p, streamE.id());
assertNotNull(p);
assertEquals(client.connectionStream().id(), p.parent().id());
assertEquals(0, p.numChildren());
}
@Test(expected = Http2Exception.class)
public void priorityChangeWithNoPrioritizableDependentsShouldRestructureTree() throws Exception {
Http2Stream streamA = client.local().createStream(1, false);
Http2Stream streamB = client.local().createStream(3, false);
Http2Stream streamC = client.local().createStream(5, false);
Http2Stream streamD = client.local().createStream(7, false);
Http2Stream streamE = client.local().createStream(9, false);
Http2Stream streamF = client.local().createStream(11, false);
streamB.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamC.setPriority(streamB.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamD.setPriority(streamB.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamE.setPriority(streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
streamF.setPriority(streamD.id(), DEFAULT_PRIORITY_WEIGHT, false);
// Leave leaf nodes open (E & F)
streamA.close();
streamB.close();
streamC.close();
streamD.close();
// Attempt to move F to depend on C, however this should throw an exception because C is closed.
streamF.setPriority(streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
}
@Test
public void circularDependencyShouldRestructureTree() throws Exception {
// Using example from http://tools.ietf.org/html/draft-ietf-httpbis-http2-16#section-5.3.3
// Initialize all the nodes
Http2Stream streamA = client.local().createStream(1, false);
verifyParentChanged(streamA, null);
Http2Stream streamB = client.local().createStream(3, false);
verifyParentChanged(streamB, null);
Http2Stream streamC = client.local().createStream(5, false);
verifyParentChanged(streamC, null);
Http2Stream streamD = client.local().createStream(7, false);
verifyParentChanged(streamD, null);
Http2Stream streamE = client.local().createStream(9, false);
verifyParentChanged(streamE, null);
Http2Stream streamF = client.local().createStream(11, false);
verifyParentChanged(streamF, null);
// Build the tree
streamB.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
verify(clientListener, never()).onWeightChanged(eq(streamB), anyShort());
verifyParentChanged(streamB, client.connectionStream());
verifyParentChanging(streamB, client.connectionStream());
streamC.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
verify(clientListener, never()).onWeightChanged(eq(streamC), anyShort());
verifyParentChanged(streamC, client.connectionStream());
verifyParentChanging(streamC, client.connectionStream());
streamD.setPriority(streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
verify(clientListener, never()).onWeightChanged(eq(streamD), anyShort());
verifyParentChanged(streamD, client.connectionStream());
verifyParentChanging(streamD, client.connectionStream());
streamE.setPriority(streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
verify(clientListener, never()).onWeightChanged(eq(streamE), anyShort());
verifyParentChanged(streamE, client.connectionStream());
verifyParentChanging(streamE, client.connectionStream());
streamF.setPriority(streamD.id(), DEFAULT_PRIORITY_WEIGHT, false);
verify(clientListener, never()).onWeightChanged(eq(streamF), anyShort());
verifyParentChanged(streamF, client.connectionStream());
verifyParentChanging(streamF, client.connectionStream());
assertEquals(6, client.numActiveStreams());
// Non-exclusive re-prioritization of a->d.
reset(clientListener);
streamA.setPriority(streamD.id(), DEFAULT_PRIORITY_WEIGHT, false);
verify(clientListener, never()).onWeightChanged(eq(streamA), anyShort());
verifyParentChanging(Arrays.asList(streamD, streamA), Arrays.asList(client.connectionStream(), streamD));
verifyParentsChanged(Arrays.asList(streamD, streamA), Arrays.asList(streamC, client.connectionStream()));
// Level 0
Http2Stream p = client.connectionStream();
assertEquals(1, p.numChildren());
// Level 1
p = child(p, streamD.id());
assertNotNull(p);
assertEquals(2, p.numChildren());
// Level 2
p = child(p, streamF.id());
assertNotNull(p);
assertEquals(0, p.numChildren());
p = child(p.parent(), streamA.id());
assertNotNull(p);
assertEquals(2, p.numChildren());
// Level 3
p = child(p, streamB.id());
assertNotNull(p);
assertEquals(0, p.numChildren());
p = child(p.parent(), streamC.id());
assertNotNull(p);
assertEquals(1, p.numChildren());
// Level 4;
p = child(p, streamE.id());
assertNotNull(p);
assertEquals(0, p.numChildren());
}
@Test
public void circularDependencyWithExclusiveShouldRestructureTree() throws Exception {
// Using example from http://tools.ietf.org/html/draft-ietf-httpbis-http2-16#section-5.3.3
// Initialize all the nodes
Http2Stream streamA = client.local().createStream(1, false);
verifyParentChanged(streamA, null);
Http2Stream streamB = client.local().createStream(3, false);
verifyParentChanged(streamB, null);
Http2Stream streamC = client.local().createStream(5, false);
verifyParentChanged(streamC, null);
Http2Stream streamD = client.local().createStream(7, false);
verifyParentChanged(streamD, null);
Http2Stream streamE = client.local().createStream(9, false);
verifyParentChanged(streamE, null);
Http2Stream streamF = client.local().createStream(11, false);
verifyParentChanged(streamF, null);
// Build the tree
streamB.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
verify(clientListener, never()).onWeightChanged(eq(streamB), anyShort());
verifyParentChanged(streamB, client.connectionStream());
verifyParentChanging(streamB, client.connectionStream());
streamC.setPriority(streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
verify(clientListener, never()).onWeightChanged(eq(streamC), anyShort());
verifyParentChanged(streamC, client.connectionStream());
verifyParentChanging(streamC, client.connectionStream());
streamD.setPriority(streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
verify(clientListener, never()).onWeightChanged(eq(streamD), anyShort());
verifyParentChanged(streamD, client.connectionStream());
verifyParentChanging(streamD, client.connectionStream());
streamE.setPriority(streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
verify(clientListener, never()).onWeightChanged(eq(streamE), anyShort());
verifyParentChanged(streamE, client.connectionStream());
verifyParentChanging(streamE, client.connectionStream());
streamF.setPriority(streamD.id(), DEFAULT_PRIORITY_WEIGHT, false);
verify(clientListener, never()).onWeightChanged(eq(streamF), anyShort());
verifyParentChanged(streamF, client.connectionStream());
verifyParentChanging(streamF, client.connectionStream());
assertEquals(6, client.numActiveStreams());
// Exclusive re-prioritization of a->d.
reset(clientListener);
streamA.setPriority(streamD.id(), DEFAULT_PRIORITY_WEIGHT, true);
verify(clientListener, never()).onWeightChanged(eq(streamA), anyShort());
verifyParentChanging(Arrays.asList(streamD, streamA, streamF),
Arrays.asList(client.connectionStream(), streamD, streamA));
verifyParentsChanged(Arrays.asList(streamD, streamA, streamF),
Arrays.asList(streamC, client.connectionStream(), streamD));
// Level 0
Http2Stream p = client.connectionStream();
assertEquals(1, p.numChildren());
// Level 1
p = child(p, streamD.id());
assertNotNull(p);
assertEquals(1, p.numChildren());
// Level 2
p = child(p, streamA.id());
assertNotNull(p);
assertEquals(3, p.numChildren());
// Level 3
p = child(p, streamB.id());
assertNotNull(p);
assertEquals(0, p.numChildren());
p = child(p.parent(), streamF.id());
assertNotNull(p);
assertEquals(0, p.numChildren());
p = child(p.parent(), streamC.id());
assertNotNull(p);
assertEquals(1, p.numChildren());
// Level 4;
p = child(p, streamE.id());
assertNotNull(p);
assertEquals(0, p.numChildren());
}
/**
* We force {@link #clientListener} methods to all throw a {@link RuntimeException} and verify the following:
* <ol>
@ -1067,7 +482,7 @@ public class DefaultHttp2ConnectionTest {
@Test
public void listenerThrowShouldNotPreventOtherListenersFromBeingNotified() throws Http2Exception {
final boolean[] calledArray = new boolean[128];
// The following setup will ensure that clienListener throws exceptions, and marks a value in an array
// The following setup will ensure that clientListener throws exceptions, and marks a value in an array
// such that clientListener2 will verify that is is set or fail the test.
int methodIndex = 0;
doAnswer(new ListenerExceptionThrower(calledArray, methodIndex))
@ -1095,21 +510,6 @@ public class DefaultHttp2ConnectionTest {
doAnswer(new ListenerVerifyCallAnswer(calledArray, methodIndex++))
.when(clientListener2).onStreamRemoved(any(Http2Stream.class));
doAnswer(new ListenerExceptionThrower(calledArray, methodIndex))
.when(clientListener).onPriorityTreeParentChanged(any(Http2Stream.class), any(Http2Stream.class));
doAnswer(new ListenerVerifyCallAnswer(calledArray, methodIndex++))
.when(clientListener2).onPriorityTreeParentChanged(any(Http2Stream.class), any(Http2Stream.class));
doAnswer(new ListenerExceptionThrower(calledArray, methodIndex))
.when(clientListener).onPriorityTreeParentChanging(any(Http2Stream.class), any(Http2Stream.class));
doAnswer(new ListenerVerifyCallAnswer(calledArray, methodIndex++))
.when(clientListener2).onPriorityTreeParentChanging(any(Http2Stream.class), any(Http2Stream.class));
doAnswer(new ListenerExceptionThrower(calledArray, methodIndex))
.when(clientListener).onWeightChanged(any(Http2Stream.class), anyShort());
doAnswer(new ListenerVerifyCallAnswer(calledArray, methodIndex++))
.when(clientListener2).onWeightChanged(any(Http2Stream.class), anyShort());
doAnswer(new ListenerExceptionThrower(calledArray, methodIndex))
.when(clientListener).onGoAwaySent(anyInt(), anyLong(), any(ByteBuf.class));
doAnswer(new ListenerVerifyCallAnswer(calledArray, methodIndex++))
@ -1125,30 +525,22 @@ public class DefaultHttp2ConnectionTest {
doAnswer(new ListenerVerifyCallAnswer(calledArray, methodIndex++))
.when(clientListener2).onStreamAdded(any(Http2Stream.class));
// Now we add clienListener2 and exercise all listener functionality
// Now we add clientListener2 and exercise all listener functionality
try {
client.addListener(clientListener2);
Http2Stream stream = client.local().createIdleStream(3);
Http2Stream stream = client.local().createStream(3, false);
verify(clientListener).onStreamAdded(any(Http2Stream.class));
verify(clientListener2).onStreamAdded(any(Http2Stream.class));
verify(clientListener, never()).onStreamActive(any(Http2Stream.class));
verify(clientListener2, never()).onStreamActive(any(Http2Stream.class));
stream.open(false);
verify(clientListener).onStreamActive(any(Http2Stream.class));
verify(clientListener2).onStreamActive(any(Http2Stream.class));
stream.setPriority(0, (short) (stream.weight() + 1), true);
verify(clientListener).onWeightChanged(any(Http2Stream.class), anyShort());
verify(clientListener2).onWeightChanged(any(Http2Stream.class), anyShort());
verify(clientListener).onPriorityTreeParentChanged(any(Http2Stream.class),
any(Http2Stream.class));
verify(clientListener2).onPriorityTreeParentChanged(any(Http2Stream.class),
any(Http2Stream.class));
verify(clientListener).onPriorityTreeParentChanging(any(Http2Stream.class),
any(Http2Stream.class));
verify(clientListener2).onPriorityTreeParentChanging(any(Http2Stream.class),
any(Http2Stream.class));
Http2Stream reservedStream = client.remote().reservePushStream(2, stream);
verify(clientListener, never()).onStreamActive(streamEq(reservedStream));
verify(clientListener2, never()).onStreamActive(streamEq(reservedStream));
reservedStream.open(false);
verify(clientListener).onStreamActive(streamEq(reservedStream));
verify(clientListener2).onStreamActive(streamEq(reservedStream));
stream.closeLocalSide();
verify(clientListener).onStreamHalfClosed(any(Http2Stream.class));
@ -1239,85 +631,8 @@ public class DefaultHttp2ConnectionTest {
}
}
private void verifyParentChanging(List<Http2Stream> expectedArg1, List<Http2Stream> expectedArg2) {
assertSame(expectedArg1.size(), expectedArg2.size());
ArgumentCaptor<Http2Stream> arg1Captor = ArgumentCaptor.forClass(Http2Stream.class);
ArgumentCaptor<Http2Stream> arg2Captor = ArgumentCaptor.forClass(Http2Stream.class);
verify(clientListener, times(expectedArg1.size())).onPriorityTreeParentChanging(arg1Captor.capture(),
arg2Captor.capture());
List<Http2Stream> capturedArg1 = arg1Captor.getAllValues();
List<Http2Stream> capturedArg2 = arg2Captor.getAllValues();
assertSame(capturedArg1.size(), capturedArg2.size());
assertSame(capturedArg1.size(), expectedArg1.size());
for (int i = 0; i < capturedArg1.size(); ++i) {
assertEquals(expectedArg1.get(i), capturedArg1.get(i));
assertEquals(expectedArg2.get(i), capturedArg2.get(i));
}
}
private void verifyParentsChanged(List<Http2Stream> expectedArg1, List<Http2Stream> expectedArg2) {
assertSame(expectedArg1.size(), expectedArg2.size());
ArgumentCaptor<Http2Stream> arg1Captor = ArgumentCaptor.forClass(Http2Stream.class);
ArgumentCaptor<Http2Stream> arg2Captor = ArgumentCaptor.forClass(Http2Stream.class);
verify(clientListener, times(expectedArg1.size())).onPriorityTreeParentChanged(arg1Captor.capture(),
arg2Captor.capture());
List<Http2Stream> capturedArg1 = arg1Captor.getAllValues();
List<Http2Stream> capturedArg2 = arg2Captor.getAllValues();
assertSame(capturedArg1.size(), capturedArg2.size());
assertSame(capturedArg1.size(), expectedArg1.size());
for (int i = 0; i < capturedArg1.size(); ++i) {
assertEquals(expectedArg1.get(i), capturedArg1.get(i));
assertEquals(expectedArg2.get(i), capturedArg2.get(i));
}
}
private static void verifyDependUponIdleStream(final Http2Stream streamA, Http2Stream streamB, Endpoint<?> endpoint)
throws Http2Exception {
assertNotNull(streamB);
assertEquals(streamB.id(), endpoint.lastStreamCreated());
assertEquals(State.IDLE, streamB.state());
assertEquals(MIN_WEIGHT, streamA.weight());
assertEquals(DEFAULT_PRIORITY_WEIGHT, streamB.weight());
assertEquals(streamB, streamA.parent());
assertEquals(1, streamB.numChildren());
streamB.forEachChild(new Http2StreamVisitor() {
@Override
public boolean visit(Http2Stream stream) throws Http2Exception {
assertEquals(streamA, stream);
return false;
}
});
}
@SuppressWarnings("unchecked")
private static <T> T streamEq(T stream) {
return (T) (stream == null ? isNull(Http2Stream.class) : eq(stream));
}
private void verifyParentChanging(Http2Stream stream, Http2Stream newParent) {
verify(clientListener).onPriorityTreeParentChanging(streamEq(stream), streamEq(newParent));
}
private void verifyParentChanged(Http2Stream stream, Http2Stream oldParent) {
verify(clientListener).onPriorityTreeParentChanged(streamEq(stream), streamEq(oldParent));
}
private Http2Stream child(Http2Stream parent, final int id) {
try {
final AtomicReference<Http2Stream> streamReference = new AtomicReference<Http2Stream>();
parent.forEachChild(new Http2StreamVisitor() {
@Override
public boolean visit(Http2Stream stream) throws Http2Exception {
if (stream.id() == id) {
streamReference.set(stream);
return false;
}
return true;
}
});
return streamReference.get();
} catch (Http2Exception e) {
PlatformDependent.throwException(e);
return null;
}
}
}

View File

@ -15,6 +15,21 @@
package io.netty.handler.codec.http2;
import io.netty.channel.Channel;
import io.netty.channel.ChannelConfig;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelPromise;
import io.netty.util.concurrent.EventExecutor;
import junit.framework.AssertionFailedError;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mock;
import org.mockito.MockitoAnnotations;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.util.concurrent.atomic.AtomicInteger;
import static io.netty.handler.codec.http2.Http2CodecUtil.CONNECTION_STREAM_ID;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_WINDOW_SIZE;
@ -38,25 +53,6 @@ import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.verifyZeroInteractions;
import static org.mockito.Mockito.when;
import io.netty.buffer.ByteBuf;
import io.netty.channel.Channel;
import io.netty.channel.ChannelConfig;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelPromise;
import io.netty.handler.codec.http2.Http2FrameWriter.Configuration;
import io.netty.util.concurrent.EventExecutor;
import junit.framework.AssertionFailedError;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mock;
import org.mockito.MockitoAnnotations;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.util.concurrent.atomic.AtomicInteger;
/**
* Tests for {@link DefaultHttp2RemoteFlowController}.
*/
@ -68,15 +64,6 @@ public abstract class DefaultHttp2RemoteFlowControllerTest {
private DefaultHttp2RemoteFlowController controller;
@Mock
private ByteBuf buffer;
@Mock
private Http2FrameSizePolicy frameWriterSizePolicy;
@Mock
private Configuration frameWriterConfiguration;
@Mock
private ChannelHandlerContext ctx;
@ -127,8 +114,8 @@ public abstract class DefaultHttp2RemoteFlowControllerTest {
connection.local().createStream(STREAM_B, false);
Http2Stream streamC = connection.local().createStream(STREAM_C, false);
Http2Stream streamD = connection.local().createStream(STREAM_D, false);
streamC.setPriority(STREAM_A, DEFAULT_PRIORITY_WEIGHT, false);
streamD.setPriority(STREAM_A, DEFAULT_PRIORITY_WEIGHT, false);
controller.updateDependencyTree(streamC.id(), STREAM_A, DEFAULT_PRIORITY_WEIGHT, false);
controller.updateDependencyTree(streamD.id(), STREAM_A, DEFAULT_PRIORITY_WEIGHT, false);
}
@Test

View File

@ -346,6 +346,72 @@ public class Http2ConnectionRoundtripTest {
verify(clientListener, never()).onRstStreamRead(any(ChannelHandlerContext.class), anyInt(), anyLong());
}
@Test
public void priorityUsingHigherValuedStreamIdDoesNotPreventUsingLowerStreamId() throws Exception {
bootstrapEnv(1, 1, 2, 0);
final Http2Headers headers = dummyHeaders();
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() throws Http2Exception {
http2Client.encoder().writePriority(ctx(), 5, 3, (short) 14, false, newPromise());
http2Client.encoder().writeHeaders(ctx(), 3, headers, 0, (short) 16, false, 0, false,
newPromise());
http2Client.flush(ctx());
}
});
assertTrue(serverSettingsAckLatch.await(5, SECONDS));
assertTrue(requestLatch.await(5, SECONDS));
verify(serverListener).onPriorityRead(any(ChannelHandlerContext.class), eq(5), eq(3), eq((short) 14),
eq(false));
verify(serverListener).onHeadersRead(any(ChannelHandlerContext.class), eq(3), eq(headers), eq(0),
eq((short) 16), eq(false), eq(0), eq(false));
// Verify that no errors have been received.
verify(serverListener, never()).onGoAwayRead(any(ChannelHandlerContext.class), anyInt(), anyLong(),
any(ByteBuf.class));
verify(serverListener, never()).onRstStreamRead(any(ChannelHandlerContext.class), anyInt(), anyLong());
verify(clientListener, never()).onGoAwayRead(any(ChannelHandlerContext.class), anyInt(), anyLong(),
any(ByteBuf.class));
verify(clientListener, never()).onRstStreamRead(any(ChannelHandlerContext.class), anyInt(), anyLong());
}
@Test
public void headersUsingHigherValuedStreamIdPreventsUsingLowerStreamId() throws Exception {
bootstrapEnv(1, 1, 1, 0);
final Http2Headers headers = dummyHeaders();
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() throws Http2Exception {
http2Client.encoder().writeHeaders(ctx(), 5, headers, 0, (short) 16, false, 0, false,
newPromise());
http2Client.encoder().frameWriter().writeHeaders(ctx(), 3, headers, 0, (short) 16, false, 0, false,
newPromise());
http2Client.flush(ctx());
}
});
assertTrue(serverSettingsAckLatch.await(5, SECONDS));
assertTrue(requestLatch.await(5, SECONDS));
verify(serverListener).onHeadersRead(any(ChannelHandlerContext.class), eq(5), eq(headers), eq(0),
eq((short) 16), eq(false), eq(0), eq(false));
verify(serverListener, never()).onHeadersRead(any(ChannelHandlerContext.class), eq(3), any(Http2Headers.class),
anyInt(), anyShort(), anyBoolean(), anyInt(), anyBoolean());
// Client should receive a RST_STREAM for stream 3, but there is not Http2Stream object so the listener is never
// notified.
verify(serverListener, never()).onGoAwayRead(any(ChannelHandlerContext.class), anyInt(), anyLong(),
any(ByteBuf.class));
verify(serverListener, never()).onRstStreamRead(any(ChannelHandlerContext.class), anyInt(), anyLong());
verify(clientListener, never()).onGoAwayRead(any(ChannelHandlerContext.class), anyInt(), anyLong(),
any(ByteBuf.class));
verify(clientListener, never()).onRstStreamRead(any(ChannelHandlerContext.class), anyInt(), anyLong());
}
@Test
public void http2ExceptionInPipelineShouldCloseConnection() throws Exception {
bootstrapEnv(1, 1, 2, 1);

View File

@ -213,9 +213,6 @@ public final class Http2TestUtil {
int streamDependency, short weight, boolean exclusive, int padding, boolean endStream)
throws Http2Exception {
Http2Stream stream = getOrCreateStream(streamId, endStream);
if (stream != null) {
stream.setPriority(streamDependency, weight, exclusive);
}
listener.onHeadersRead(ctx, streamId, headers, streamDependency, weight, exclusive, padding,
endStream);
if (endStream) {
@ -227,10 +224,6 @@ public final class Http2TestUtil {
@Override
public void onPriorityRead(ChannelHandlerContext ctx, int streamId, int streamDependency, short weight,
boolean exclusive) throws Http2Exception {
Http2Stream stream = getOrCreateStream(streamId, false);
if (stream != null) {
stream.setPriority(streamDependency, weight, exclusive);
}
listener.onPriorityRead(ctx, streamId, streamDependency, weight, exclusive);
latch.countDown();
}

View File

@ -68,7 +68,7 @@ import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
/**
* Testing the {@link InboundHttp2ToHttpPriorityAdapter} and base class {@link InboundHttp2ToHttpAdapter} for HTTP/2
* Testing the {@link InboundHttp2ToHttpAdapter} and base class {@link InboundHttp2ToHttpAdapter} for HTTP/2
* frames into {@link HttpObject}s
*/
public class InboundHttp2ToHttpAdapterTest {
@ -473,9 +473,9 @@ public class InboundHttp2ToHttpAdapterTest {
@Override
public void run() throws Http2Exception {
clientHandler.encoder().writeHeaders(ctxClient(), 3, http2Headers, 0, false, newPromiseClient());
clientHandler.encoder().writeHeaders(ctxClient(), 5, http2Headers2, 0, false, newPromiseClient());
clientHandler.encoder().writeHeaders(ctxClient(), 5, http2Headers2, 3, (short) 123, true, 0,
false, newPromiseClient());
clientChannel.flush(); // Headers are queued in the flow controller and so flush them.
clientHandler.encoder().writePriority(ctxClient(), 5, 3, (short) 123, true, newPromiseClient());
clientHandler.encoder().writeData(ctxClient(), 3, content.retainedDuplicate(), 0, true,
newPromiseClient());
clientHandler.encoder().writeData(ctxClient(), 5, content2.retainedDuplicate(), 0, true,
@ -495,66 +495,6 @@ public class InboundHttp2ToHttpAdapterTest {
}
}
@Test
public void clientRequestStreamDependencyOutsideHttpMessageFlow() throws Exception {
boostrapEnv(1, 3, 1);
final String text = "hello world big time data!";
final ByteBuf content = Unpooled.copiedBuffer(text.getBytes());
final String text2 = "hello world big time data...number 2!!";
final ByteBuf content2 = Unpooled.copiedBuffer(text2.getBytes());
final FullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.PUT,
"/some/path/resource", content, true);
final FullHttpMessage request2 = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.PUT,
"/some/path/resource2", content2, true);
final FullHttpMessage request3 = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1,
HttpConversionUtil.OUT_OF_MESSAGE_SEQUENCE_METHOD,
HttpConversionUtil.OUT_OF_MESSAGE_SEQUENCE_PATH, true);
try {
HttpHeaders httpHeaders = request.headers();
httpHeaders.setInt(HttpConversionUtil.ExtensionHeaderNames.STREAM_ID.text(), 3);
httpHeaders.setInt(HttpHeaderNames.CONTENT_LENGTH, text.length());
httpHeaders.setShort(HttpConversionUtil.ExtensionHeaderNames.STREAM_WEIGHT.text(), (short) 16);
HttpHeaders httpHeaders2 = request2.headers();
httpHeaders2.setInt(HttpConversionUtil.ExtensionHeaderNames.STREAM_ID.text(), 5);
httpHeaders2.setInt(HttpHeaderNames.CONTENT_LENGTH, text2.length());
httpHeaders2.setShort(HttpConversionUtil.ExtensionHeaderNames.STREAM_WEIGHT.text(), (short) 16);
final Http2Headers http2Headers = new DefaultHttp2Headers().method(new AsciiString("PUT")).path(
new AsciiString("/some/path/resource"));
final Http2Headers http2Headers2 = new DefaultHttp2Headers().method(new AsciiString("PUT")).path(
new AsciiString("/some/path/resource2"));
HttpHeaders httpHeaders3 = request3.headers();
httpHeaders3.setInt(HttpConversionUtil.ExtensionHeaderNames.STREAM_ID.text(), 5);
httpHeaders3.setInt(HttpConversionUtil.ExtensionHeaderNames.STREAM_DEPENDENCY_ID.text(), 3);
httpHeaders3.setShort(HttpConversionUtil.ExtensionHeaderNames.STREAM_WEIGHT.text(), (short) 222);
httpHeaders3.setInt(HttpHeaderNames.CONTENT_LENGTH, 0);
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() throws Http2Exception {
clientHandler.encoder().writeHeaders(ctxClient(), 3, http2Headers, 0, false, newPromiseClient());
clientHandler.encoder().writeHeaders(ctxClient(), 5, http2Headers2, 0, false, newPromiseClient());
clientHandler.encoder().writeData(ctxClient(), 3, content.retainedDuplicate(), 0, true,
newPromiseClient());
clientHandler.encoder().writeData(ctxClient(), 5, content2.retainedDuplicate(), 0, true,
newPromiseClient());
clientChannel.flush(); // headers and data are queued in the flow controller, so flush them.
clientHandler.encoder().writePriority(ctxClient(), 5, 3, (short) 222, false, newPromiseClient());
clientChannel.flush();
}
});
awaitRequests();
ArgumentCaptor<FullHttpMessage> httpObjectCaptor = ArgumentCaptor.forClass(FullHttpMessage.class);
verify(serverListener, times(3)).messageReceived(httpObjectCaptor.capture());
capturedRequests = httpObjectCaptor.getAllValues();
assertEquals(request, capturedRequests.get(0));
assertEquals(request2, capturedRequests.get(1));
assertEquals(request3, capturedRequests.get(2));
} finally {
request.release();
request2.release();
request3.release();
}
}
@Test
public void serverRequestPushPromise() throws Exception {
boostrapEnv(1, 1, 1);
@ -771,7 +711,7 @@ public class InboundHttp2ToHttpAdapterTest {
Http2Connection connection = new DefaultHttp2Connection(true);
serverHandler = new Http2ConnectionHandlerBuilder().frameListener(
new InboundHttp2ToHttpPriorityAdapterBuilder(connection)
new InboundHttp2ToHttpAdapterBuilder(connection)
.maxContentLength(maxContentLength)
.validateHttpHeaders(true)
.propagateSettings(true)
@ -798,7 +738,7 @@ public class InboundHttp2ToHttpAdapterTest {
Http2Connection connection = new DefaultHttp2Connection(false);
clientHandler = new Http2ConnectionHandlerBuilder().frameListener(
new InboundHttp2ToHttpPriorityAdapterBuilder(connection)
new InboundHttp2ToHttpAdapterBuilder(connection)
.maxContentLength(maxContentLength)
.build())
.connection(connection)

View File

@ -461,7 +461,7 @@ public class StreamBufferingEncoderTest {
@Test
public void closeShouldCancelAllBufferedStreams() throws Http2Exception {
encoder.writeSettingsAck(ctx, newPromise());
connection.local().maxStreams(0, 0);
connection.local().maxActiveStreams(0);
ChannelFuture f1 = encoderWriteHeaders(3, newPromise());
ChannelFuture f2 = encoderWriteHeaders(5, newPromise());

View File

@ -14,8 +14,17 @@
*/
package io.netty.handler.codec.http2;
import org.junit.Before;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.mockito.Mock;
import org.mockito.MockitoAnnotations;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.mockito.verification.VerificationMode;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_MIN_ALLOCATION_CHUNK;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT;
import static io.netty.handler.codec.http2.UniformStreamByteDistributor.DEFAULT_MIN_ALLOCATION_CHUNK;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertSame;
@ -33,15 +42,6 @@ import static org.mockito.Mockito.reset;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import org.junit.Before;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.mockito.Mock;
import org.mockito.MockitoAnnotations;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.mockito.verification.VerificationMode;
/**
* Tests for {@link UniformStreamByteDistributor}.
*/
@ -73,8 +73,8 @@ public class UniformStreamByteDistributorTest {
connection.local().createStream(STREAM_B, false);
Http2Stream streamC = connection.local().createStream(STREAM_C, false);
Http2Stream streamD = connection.local().createStream(STREAM_D, false);
streamC.setPriority(STREAM_A, DEFAULT_PRIORITY_WEIGHT, false);
streamD.setPriority(STREAM_A, DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamC.id(), STREAM_A, DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamD.id(), STREAM_A, DEFAULT_PRIORITY_WEIGHT, false);
}
private Answer<Void> writeAnswer() {
@ -259,7 +259,7 @@ public class UniformStreamByteDistributorTest {
}
private void setPriority(int streamId, int parent, int weight, boolean exclusive) throws Http2Exception {
stream(streamId).setPriority(parent, (short) weight, exclusive);
distributor.updateDependencyTree(streamId, parent, (short) weight, exclusive);
}
private boolean write(int numBytes) throws Http2Exception {

View File

@ -0,0 +1,902 @@
/*
* Copyright 2017 The Netty Project
*
* The Netty Project 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 io.netty.handler.codec.http2;
import org.junit.Before;
import org.junit.Test;
import org.mockito.MockitoAnnotations;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_WEIGHT;
import static io.netty.handler.codec.http2.Http2CodecUtil.MIN_WEIGHT;
import static io.netty.handler.codec.http2.WeightedFairQueueByteDistributor.INITIAL_CHILDREN_MAP_SIZE;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyInt;
import static org.mockito.Mockito.doAnswer;
public class WeightedFairQueueByteDistributorDependencyTreeTest extends
AbstractWeightedFairQueueByteDistributorDependencyTest {
@Before
public void setup() throws Http2Exception {
MockitoAnnotations.initMocks(this);
setup(0);
}
private void setup(int maxStateOnlySize) {
connection = new DefaultHttp2Connection(false);
distributor = new WeightedFairQueueByteDistributor(connection, maxStateOnlySize);
// Assume we always write all the allocated bytes.
doAnswer(writeAnswer(false)).when(writer).write(any(Http2Stream.class), anyInt());
}
@Test
public void closingStreamWithChildrenDoesNotCauseConcurrentModification() throws Http2Exception {
// We create enough streams to wrap around the child array. We carefully craft the stream ids so that they hash
// codes overlap with respect to the child collection. If the implementation is not careful this may lead to a
// concurrent modification exception while promoting all children to the connection stream.
final Http2Stream streamA = connection.local().createStream(1, false);
final int numStreams = INITIAL_CHILDREN_MAP_SIZE - 1;
for (int i = 0, streamId = 3; i < numStreams; ++i, streamId += INITIAL_CHILDREN_MAP_SIZE) {
final Http2Stream stream = connection.local().createStream(streamId, false);
setPriority(stream.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
}
assertEquals(INITIAL_CHILDREN_MAP_SIZE, connection.numActiveStreams());
streamA.close();
assertEquals(numStreams, connection.numActiveStreams());
}
@Test
public void closeWhileIteratingDoesNotNPE() throws Http2Exception {
final Http2Stream streamA = connection.local().createStream(3, false);
final Http2Stream streamB = connection.local().createStream(5, false);
final Http2Stream streamC = connection.local().createStream(7, false);
setPriority(streamB.id(), streamA.id(), Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT, false);
connection.forEachActiveStream(new Http2StreamVisitor() {
@Override
public boolean visit(Http2Stream stream) throws Http2Exception {
streamA.close();
setPriority(streamB.id(), streamC.id(), Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT, false);
return true;
}
});
}
@Test
public void localStreamCanDependUponIdleStream() throws Http2Exception {
setup(1);
Http2Stream streamA = connection.local().createStream(1, false);
setPriority(3, streamA.id(), MIN_WEIGHT, true);
assertTrue(distributor.isChild(3, streamA.id(), MIN_WEIGHT));
}
@Test
public void remoteStreamCanDependUponIdleStream() throws Http2Exception {
setup(1);
Http2Stream streamA = connection.remote().createStream(2, false);
setPriority(4, streamA.id(), MIN_WEIGHT, true);
assertTrue(distributor.isChild(4, streamA.id(), MIN_WEIGHT));
}
@Test
public void prioritizeShouldUseDefaults() throws Exception {
Http2Stream stream = connection.local().createStream(1, false);
assertTrue(distributor.isChild(stream.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(1, distributor.numChildren(connection.connectionStream().id()));
assertEquals(0, distributor.numChildren(stream.id()));
}
@Test
public void reprioritizeWithNoChangeShouldDoNothing() throws Exception {
Http2Stream stream = connection.local().createStream(1, false);
setPriority(stream.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT, false);
assertTrue(distributor.isChild(stream.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(1, distributor.numChildren(connection.connectionStream().id()));
assertEquals(0, distributor.numChildren(stream.id()));
}
@Test
public void stateOnlyPriorityShouldBePreservedWhenStreamsAreCreatedAndClosed() throws Http2Exception {
setup(3);
short weight3 = MIN_WEIGHT + 1;
short weight5 = (short) (weight3 + 1);
short weight7 = (short) (weight5 + 1);
setPriority(3, connection.connectionStream().id(), weight3, true);
setPriority(5, connection.connectionStream().id(), weight5, true);
setPriority(7, connection.connectionStream().id(), weight7, true);
assertEquals(0, connection.numActiveStreams());
verifyStateOnlyPriorityShouldBePreservedWhenStreamsAreCreated(weight3, weight5, weight7);
// Now create stream objects and ensure the state and dependency tree is preserved.
Http2Stream streamA = connection.local().createStream(3, false);
Http2Stream streamB = connection.local().createStream(5, false);
Http2Stream streamC = connection.local().createStream(7, false);
assertEquals(3, connection.numActiveStreams());
verifyStateOnlyPriorityShouldBePreservedWhenStreamsAreCreated(weight3, weight5, weight7);
// Close all the streams and ensure the state and dependency tree is preserved.
streamA.close();
streamB.close();
streamC.close();
assertEquals(0, connection.numActiveStreams());
verifyStateOnlyPriorityShouldBePreservedWhenStreamsAreCreated(weight3, weight5, weight7);
}
private void verifyStateOnlyPriorityShouldBePreservedWhenStreamsAreCreated(short weight3, short weight5,
short weight7) {
// Level 0
assertEquals(1, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(7, connection.connectionStream().id(), weight7));
assertEquals(1, distributor.numChildren(7));
// Level 2
assertTrue(distributor.isChild(5, 7, weight5));
assertEquals(1, distributor.numChildren(5));
// Level 3
assertTrue(distributor.isChild(3, 5, weight3));
assertEquals(0, distributor.numChildren(3));
}
private static final int leadersId = 3; // js, css
private static final int unblockedId = 5;
private static final int backgroundId = 7;
private static final int speculativeId = 9;
private static final int followersId = 11; // images
private static final short leadersWeight = 201;
private static final short unblockedWeight = 101;
private static final short backgroundWeight = 1;
private static final short speculativeWeight = 1;
private static final short followersWeight = 1;
@Test
public void fireFoxQoSStreamsRemainAfterDataStreamsAreClosed() throws Http2Exception {
// http://bitsup.blogspot.com/2015/01/http2-dependency-priorities-in-firefox.html
setup(5);
setPriority(leadersId, connection.connectionStream().id(), leadersWeight, false);
setPriority(unblockedId, connection.connectionStream().id(), unblockedWeight, false);
setPriority(backgroundId, connection.connectionStream().id(), backgroundWeight, false);
setPriority(speculativeId, backgroundId, speculativeWeight, false);
setPriority(followersId, leadersId, followersWeight, false);
verifyFireFoxQoSStreams();
// Simulate a HTML request
short htmlGetStreamWeight = 2;
Http2Stream htmlGetStream = connection.local().createStream(13, false);
setPriority(htmlGetStream.id(), followersId, htmlGetStreamWeight, false);
Http2Stream favIconStream = connection.local().createStream(15, false);
setPriority(favIconStream.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT, false);
Http2Stream cssStream = connection.local().createStream(17, false);
setPriority(cssStream.id(), leadersId, DEFAULT_PRIORITY_WEIGHT, false);
Http2Stream jsStream = connection.local().createStream(19, false);
setPriority(jsStream.id(), leadersId, DEFAULT_PRIORITY_WEIGHT, false);
Http2Stream imageStream = connection.local().createStream(21, false);
setPriority(imageStream.id(), followersId, 1, false);
// Level 0
assertEquals(4, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(leadersId, connection.connectionStream().id(), leadersWeight));
assertEquals(3, distributor.numChildren(leadersId));
assertTrue(distributor.isChild(unblockedId, connection.connectionStream().id(), unblockedWeight));
assertEquals(0, distributor.numChildren(unblockedId));
assertTrue(distributor.isChild(backgroundId, connection.connectionStream().id(), backgroundWeight));
assertEquals(1, distributor.numChildren(backgroundId));
assertTrue(distributor.isChild(favIconStream.id(), connection.connectionStream().id(),
DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(favIconStream.id()));
// Level 2
assertTrue(distributor.isChild(followersId, leadersId, followersWeight));
assertEquals(2, distributor.numChildren(followersId));
assertTrue(distributor.isChild(speculativeId, backgroundId, speculativeWeight));
assertEquals(0, distributor.numChildren(speculativeId));
assertTrue(distributor.isChild(cssStream.id(), leadersId, DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(cssStream.id()));
assertTrue(distributor.isChild(jsStream.id(), leadersId, DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(jsStream.id()));
// Level 3
assertTrue(distributor.isChild(htmlGetStream.id(), followersId, htmlGetStreamWeight));
assertEquals(0, distributor.numChildren(htmlGetStream.id()));
assertTrue(distributor.isChild(imageStream.id(), followersId, followersWeight));
assertEquals(0, distributor.numChildren(imageStream.id()));
// Close all the data streams and ensure the "priority only streams" are retained in the dependency tree.
htmlGetStream.close();
favIconStream.close();
cssStream.close();
jsStream.close();
imageStream.close();
verifyFireFoxQoSStreams();
}
private void verifyFireFoxQoSStreams() {
// Level 0
assertEquals(3, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(leadersId, connection.connectionStream().id(), leadersWeight));
assertEquals(1, distributor.numChildren(leadersId));
assertTrue(distributor.isChild(unblockedId, connection.connectionStream().id(), unblockedWeight));
assertEquals(0, distributor.numChildren(unblockedId));
assertTrue(distributor.isChild(backgroundId, connection.connectionStream().id(), backgroundWeight));
assertEquals(1, distributor.numChildren(backgroundId));
// Level 2
assertTrue(distributor.isChild(followersId, leadersId, followersWeight));
assertEquals(0, distributor.numChildren(followersId));
assertTrue(distributor.isChild(speculativeId, backgroundId, speculativeWeight));
assertEquals(0, distributor.numChildren(speculativeId));
}
@Test
public void lowestPrecedenceStateShouldBeDropped() throws Http2Exception {
setup(3);
short weight3 = MAX_WEIGHT;
short weight5 = (short) (weight3 - 1);
short weight7 = (short) (weight5 - 1);
short weight9 = (short) (weight7 - 1);
setPriority(3, connection.connectionStream().id(), weight3, true);
setPriority(5, connection.connectionStream().id(), weight5, true);
setPriority(7, connection.connectionStream().id(), weight7, false);
assertEquals(0, connection.numActiveStreams());
verifyLowestPrecedenceStateShouldBeDropped1(weight3, weight5, weight7);
// Attempt to create a new item in the dependency tree but the maximum amount of "state only" streams is meet
// so a stream will have to be dropped. Currently the new stream is the lowest "precedence" so it is dropped.
setPriority(9, 3, weight9, false);
assertEquals(0, connection.numActiveStreams());
verifyLowestPrecedenceStateShouldBeDropped1(weight3, weight5, weight7);
// Set the priority for stream 9 such that its depth in the dependency tree is numerically lower than stream 3,
// and therefore the dependency state associated with stream 3 will be dropped.
setPriority(9, 5, weight9, true);
verifyLowestPrecedenceStateShouldBeDropped2(weight9, weight5, weight7);
// Test that stream which has been activated is lower priority than other streams that have not been activated.
Http2Stream streamA = connection.local().createStream(5, false);
streamA.close();
verifyLowestPrecedenceStateShouldBeDropped2(weight9, weight5, weight7);
// Stream 3 (hasn't been opened) should result in stream 5 being dropped.
setPriority(3, 9, weight3, false);
verifyLowestPrecedenceStateShouldBeDropped3(weight3, weight7, weight9);
// Stream 5's state has been discarded so we should be able to re-insert this state.
setPriority(5, 0, weight5, false);
verifyLowestPrecedenceStateShouldBeDropped4(weight5, weight7, weight9);
// All streams are at the same level, so stream ID should be used to drop the numeric lowest valued stream.
short weight11 = (short) (weight9 - 1);
setPriority(11, 0, weight11, false);
verifyLowestPrecedenceStateShouldBeDropped5(weight7, weight9, weight11);
}
private void verifyLowestPrecedenceStateShouldBeDropped1(short weight3, short weight5, short weight7) {
// Level 0
assertEquals(2, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(7, connection.connectionStream().id(), weight7));
assertEquals(0, distributor.numChildren(7));
assertTrue(distributor.isChild(5, connection.connectionStream().id(), weight5));
assertEquals(1, distributor.numChildren(5));
// Level 2
assertTrue(distributor.isChild(3, 5, weight3));
assertEquals(0, distributor.numChildren(3));
}
private void verifyLowestPrecedenceStateShouldBeDropped2(short weight9, short weight5, short weight7) {
// Level 0
assertEquals(2, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(7, connection.connectionStream().id(), weight7));
assertEquals(0, distributor.numChildren(7));
assertTrue(distributor.isChild(5, connection.connectionStream().id(), weight5));
assertEquals(1, distributor.numChildren(5));
// Level 2
assertTrue(distributor.isChild(9, 5, weight9));
assertEquals(0, distributor.numChildren(9));
}
private void verifyLowestPrecedenceStateShouldBeDropped3(short weight3, short weight7, short weight9) {
// Level 0
assertEquals(2, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(7, connection.connectionStream().id(), weight7));
assertEquals(0, distributor.numChildren(7));
assertTrue(distributor.isChild(9, connection.connectionStream().id(), weight9));
assertEquals(1, distributor.numChildren(9));
// Level 2
assertTrue(distributor.isChild(3, 9, weight3));
assertEquals(0, distributor.numChildren(3));
}
private void verifyLowestPrecedenceStateShouldBeDropped4(short weight5, short weight7, short weight9) {
// Level 0
assertEquals(3, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(5, connection.connectionStream().id(), weight5));
assertEquals(0, distributor.numChildren(5));
assertTrue(distributor.isChild(7, connection.connectionStream().id(), weight7));
assertEquals(0, distributor.numChildren(7));
assertTrue(distributor.isChild(9, connection.connectionStream().id(), weight9));
assertEquals(0, distributor.numChildren(9));
}
private void verifyLowestPrecedenceStateShouldBeDropped5(short weight7, short weight9, short weight11) {
// Level 0
assertEquals(3, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(11, connection.connectionStream().id(), weight11));
assertEquals(0, distributor.numChildren(11));
assertTrue(distributor.isChild(7, connection.connectionStream().id(), weight7));
assertEquals(0, distributor.numChildren(7));
assertTrue(distributor.isChild(9, connection.connectionStream().id(), weight9));
assertEquals(0, distributor.numChildren(9));
}
@Test
public void priorityOnlyStreamsArePreservedWhenReservedStreamsAreClosed() throws Http2Exception {
setup(1);
short weight3 = MIN_WEIGHT;
setPriority(3, connection.connectionStream().id(), weight3, true);
Http2Stream streamA = connection.local().createStream(5, false);
Http2Stream streamB = connection.remote().reservePushStream(4, streamA);
// Level 0
assertEquals(3, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(3, connection.connectionStream().id(), weight3));
assertEquals(0, distributor.numChildren(3));
assertTrue(distributor.isChild(streamA.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamA.id()));
assertTrue(distributor.isChild(streamB.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamB.id()));
// Close both streams.
streamB.close();
streamA.close();
// Level 0
assertEquals(1, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(3, connection.connectionStream().id(), weight3));
assertEquals(0, distributor.numChildren(3));
}
@Test
public void insertExclusiveShouldAddNewLevel() throws Exception {
Http2Stream streamA = connection.local().createStream(1, false);
Http2Stream streamB = connection.local().createStream(3, false);
Http2Stream streamC = connection.local().createStream(5, false);
Http2Stream streamD = connection.local().createStream(7, false);
setPriority(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamD.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, true);
assertEquals(4, connection.numActiveStreams());
// Level 0
assertEquals(1, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(streamA.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(1, distributor.numChildren(streamA.id()));
// Level 2
assertTrue(distributor.isChild(streamD.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(2, distributor.numChildren(streamD.id()));
// Level 3
assertTrue(distributor.isChild(streamB.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamB.id()));
assertTrue(distributor.isChild(streamC.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamC.id()));
}
@Test
public void existingChildMadeExclusiveShouldNotCreateTreeCycle() throws Http2Exception {
Http2Stream streamA = connection.local().createStream(1, false);
Http2Stream streamB = connection.local().createStream(3, false);
Http2Stream streamC = connection.local().createStream(5, false);
Http2Stream streamD = connection.local().createStream(7, false);
setPriority(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamD.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
// Stream C is already dependent on Stream A, but now make that an exclusive dependency
setPriority(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, true);
assertEquals(4, connection.numActiveStreams());
// Level 0
assertEquals(1, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(streamA.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(1, distributor.numChildren(streamA.id()));
// Level 2
assertTrue(distributor.isChild(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(2, distributor.numChildren(streamC.id()));
// Level 3
assertTrue(distributor.isChild(streamB.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamB.id()));
assertTrue(distributor.isChild(streamD.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamD.id()));
}
@Test
public void newExclusiveChildShouldUpdateOldParentCorrectly() throws Http2Exception {
Http2Stream streamA = connection.local().createStream(1, false);
Http2Stream streamB = connection.local().createStream(3, false);
Http2Stream streamC = connection.local().createStream(5, false);
Http2Stream streamD = connection.local().createStream(7, false);
Http2Stream streamE = connection.local().createStream(9, false);
Http2Stream streamF = connection.local().createStream(11, false);
setPriority(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamD.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamF.id(), streamE.id(), DEFAULT_PRIORITY_WEIGHT, false);
// F is now going to be exclusively dependent on A, after this we should check that stream E
// prioritizableForTree is not over decremented.
setPriority(streamF.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, true);
assertEquals(6, connection.numActiveStreams());
// Level 0
assertEquals(2, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(streamE.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamE.id()));
assertTrue(distributor.isChild(streamA.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(1, distributor.numChildren(streamA.id()));
// Level 2
assertTrue(distributor.isChild(streamF.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(2, distributor.numChildren(streamF.id()));
// Level 3
assertTrue(distributor.isChild(streamB.id(), streamF.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamB.id()));
assertTrue(distributor.isChild(streamC.id(), streamF.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(1, distributor.numChildren(streamC.id()));
// Level 4
assertTrue(distributor.isChild(streamD.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamD.id()));
}
@Test
public void weightChangeWithNoTreeChangeShouldBeRespected() throws Http2Exception {
Http2Stream streamA = connection.local().createStream(1, false);
Http2Stream streamB = connection.local().createStream(3, false);
Http2Stream streamC = connection.local().createStream(5, false);
Http2Stream streamD = connection.local().createStream(7, false);
setPriority(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamD.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, true);
assertEquals(4, connection.numActiveStreams());
short newWeight = (short) (DEFAULT_PRIORITY_WEIGHT + 1);
setPriority(streamD.id(), streamA.id(), newWeight, false);
// Level 0
assertEquals(1, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(streamA.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(1, distributor.numChildren(streamA.id()));
// Level 2
assertTrue(distributor.isChild(streamD.id(), streamA.id(), newWeight));
assertEquals(2, distributor.numChildren(streamD.id()));
// Level 3
assertTrue(distributor.isChild(streamB.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamB.id()));
assertTrue(distributor.isChild(streamC.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamC.id()));
}
@Test
public void sameNodeDependentShouldNotStackOverflowNorChangePrioritizableForTree() throws Http2Exception {
Http2Stream streamA = connection.local().createStream(1, false);
Http2Stream streamB = connection.local().createStream(3, false);
Http2Stream streamC = connection.local().createStream(5, false);
Http2Stream streamD = connection.local().createStream(7, false);
setPriority(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamD.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, true);
boolean[] exclusives = new boolean[] {true, false};
short[] weights = new short[] { DEFAULT_PRIORITY_WEIGHT/*, 100, 200, DEFAULT_PRIORITY_WEIGHT */};
assertEquals(4, connection.numActiveStreams());
// The goal is to call setPriority with the same parent and vary the parameters
// we were at one point adding a circular depends to the tree and then throwing
// a StackOverflow due to infinite recursive operation.
for (short weight : weights) {
for (boolean exclusive : exclusives) {
setPriority(streamD.id(), streamA.id(), weight, exclusive);
assertEquals(0, distributor.numChildren(streamB.id()));
assertEquals(0, distributor.numChildren(streamC.id()));
assertEquals(1, distributor.numChildren(streamA.id()));
assertEquals(2, distributor.numChildren(streamD.id()));
assertFalse(distributor.isChild(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertFalse(distributor.isChild(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertTrue(distributor.isChild(streamB.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT));
assertTrue(distributor.isChild(streamC.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT));
assertTrue(distributor.isChild(streamD.id(), streamA.id(), weight));
}
}
}
@Test
public void multipleCircularDependencyShouldUpdatePrioritizable() throws Http2Exception {
Http2Stream streamA = connection.local().createStream(1, false);
Http2Stream streamB = connection.local().createStream(3, false);
Http2Stream streamC = connection.local().createStream(5, false);
Http2Stream streamD = connection.local().createStream(7, false);
setPriority(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamD.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, true);
assertEquals(4, connection.numActiveStreams());
// Bring B to the root
setPriority(streamA.id(), streamB.id(), DEFAULT_PRIORITY_WEIGHT, true);
// Move all streams to be children of B
setPriority(streamC.id(), streamB.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamD.id(), streamB.id(), DEFAULT_PRIORITY_WEIGHT, false);
// Move A back to the root
setPriority(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, true);
// Move all streams to be children of A
setPriority(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamD.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
// Level 0
assertEquals(1, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(streamA.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(3, distributor.numChildren(streamA.id()));
// Level 2
assertTrue(distributor.isChild(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamB.id()));
assertTrue(distributor.isChild(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamC.id()));
assertTrue(distributor.isChild(streamD.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamD.id()));
}
@Test
public void removeWithPrioritizableDependentsShouldNotRestructureTree() throws Exception {
Http2Stream streamA = connection.local().createStream(1, false);
Http2Stream streamB = connection.local().createStream(3, false);
Http2Stream streamC = connection.local().createStream(5, false);
Http2Stream streamD = connection.local().createStream(7, false);
setPriority(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamC.id(), streamB.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamD.id(), streamB.id(), DEFAULT_PRIORITY_WEIGHT, false);
// Default removal policy will cause it to be removed immediately.
streamB.close();
// Level 0
assertEquals(1, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(streamA.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(2, distributor.numChildren(streamA.id()));
// Level 2
assertTrue(distributor.isChild(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamC.id()));
assertTrue(distributor.isChild(streamD.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamD.id()));
}
@Test
public void closeWithNoPrioritizableDependentsShouldRestructureTree() throws Exception {
Http2Stream streamA = connection.local().createStream(1, false);
Http2Stream streamB = connection.local().createStream(3, false);
Http2Stream streamC = connection.local().createStream(5, false);
Http2Stream streamD = connection.local().createStream(7, false);
Http2Stream streamE = connection.local().createStream(9, false);
Http2Stream streamF = connection.local().createStream(11, false);
setPriority(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamC.id(), streamB.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamD.id(), streamB.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamE.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamF.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT, false);
// Close internal nodes, leave 1 leaf node open, the only remaining stream is the one that is not closed (E).
streamA.close();
streamB.close();
streamC.close();
streamD.close();
streamF.close();
// Level 0
assertEquals(1, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(streamE.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamE.id()));
}
@Test
public void priorityChangeWithNoPrioritizableDependentsShouldRestructureTree() throws Exception {
Http2Stream streamA = connection.local().createStream(1, false);
Http2Stream streamB = connection.local().createStream(3, false);
Http2Stream streamC = connection.local().createStream(5, false);
Http2Stream streamD = connection.local().createStream(7, false);
Http2Stream streamE = connection.local().createStream(9, false);
Http2Stream streamF = connection.local().createStream(11, false);
setPriority(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamC.id(), streamB.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamD.id(), streamB.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamF.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamE.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
// Leave leaf nodes open (E & F)
streamA.close();
streamB.close();
streamC.close();
streamD.close();
// Move F to depend on C, even though C is closed.
setPriority(streamF.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
// Level 0
assertEquals(2, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(streamE.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamE.id()));
assertTrue(distributor.isChild(streamF.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamF.id()));
}
@Test
public void circularDependencyShouldRestructureTree() throws Exception {
// Using example from https://tools.ietf.org/html/rfc7540#section-5.3.3
// Initialize all the nodes
Http2Stream streamA = connection.local().createStream(1, false);
Http2Stream streamB = connection.local().createStream(3, false);
Http2Stream streamC = connection.local().createStream(5, false);
Http2Stream streamD = connection.local().createStream(7, false);
Http2Stream streamE = connection.local().createStream(9, false);
Http2Stream streamF = connection.local().createStream(11, false);
assertEquals(6, distributor.numChildren(connection.connectionStream().id()));
assertEquals(0, distributor.numChildren(streamA.id()));
assertEquals(0, distributor.numChildren(streamB.id()));
assertEquals(0, distributor.numChildren(streamC.id()));
assertEquals(0, distributor.numChildren(streamD.id()));
assertEquals(0, distributor.numChildren(streamE.id()));
assertEquals(0, distributor.numChildren(streamF.id()));
// Build the tree
setPriority(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
assertEquals(5, distributor.numChildren(connection.connectionStream().id()));
assertTrue(distributor.isChild(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(1, distributor.numChildren(streamA.id()));
setPriority(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
assertEquals(4, distributor.numChildren(connection.connectionStream().id()));
assertTrue(distributor.isChild(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(2, distributor.numChildren(streamA.id()));
setPriority(streamD.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
assertEquals(3, distributor.numChildren(connection.connectionStream().id()));
assertTrue(distributor.isChild(streamD.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(1, distributor.numChildren(streamC.id()));
setPriority(streamE.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
assertEquals(2, distributor.numChildren(connection.connectionStream().id()));
assertTrue(distributor.isChild(streamE.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(2, distributor.numChildren(streamC.id()));
setPriority(streamF.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT, false);
assertEquals(1, distributor.numChildren(connection.connectionStream().id()));
assertTrue(distributor.isChild(streamF.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(1, distributor.numChildren(streamD.id()));
assertEquals(6, connection.numActiveStreams());
// Non-exclusive re-prioritization of a->d.
setPriority(streamA.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT, false);
// Level 0
assertEquals(1, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(streamD.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(2, distributor.numChildren(streamD.id()));
// Level 2
assertTrue(distributor.isChild(streamF.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamF.id()));
assertTrue(distributor.isChild(streamA.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(2, distributor.numChildren(streamA.id()));
// Level 3
assertTrue(distributor.isChild(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamB.id()));
assertTrue(distributor.isChild(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(1, distributor.numChildren(streamC.id()));
// Level 4
assertTrue(distributor.isChild(streamE.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamE.id()));
}
@Test
public void circularDependencyWithExclusiveShouldRestructureTree() throws Exception {
// Using example from https://tools.ietf.org/html/rfc7540#section-5.3.3
// Initialize all the nodes
Http2Stream streamA = connection.local().createStream(1, false);
Http2Stream streamB = connection.local().createStream(3, false);
Http2Stream streamC = connection.local().createStream(5, false);
Http2Stream streamD = connection.local().createStream(7, false);
Http2Stream streamE = connection.local().createStream(9, false);
Http2Stream streamF = connection.local().createStream(11, false);
assertEquals(6, distributor.numChildren(connection.connectionStream().id()));
assertEquals(0, distributor.numChildren(streamA.id()));
assertEquals(0, distributor.numChildren(streamB.id()));
assertEquals(0, distributor.numChildren(streamC.id()));
assertEquals(0, distributor.numChildren(streamD.id()));
assertEquals(0, distributor.numChildren(streamE.id()));
assertEquals(0, distributor.numChildren(streamF.id()));
// Build the tree
setPriority(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
assertEquals(5, distributor.numChildren(connection.connectionStream().id()));
assertTrue(distributor.isChild(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(1, distributor.numChildren(streamA.id()));
setPriority(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT, false);
assertEquals(4, distributor.numChildren(connection.connectionStream().id()));
assertTrue(distributor.isChild(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(2, distributor.numChildren(streamA.id()));
setPriority(streamD.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
assertEquals(3, distributor.numChildren(connection.connectionStream().id()));
assertTrue(distributor.isChild(streamD.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(1, distributor.numChildren(streamC.id()));
setPriority(streamE.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT, false);
assertEquals(2, distributor.numChildren(connection.connectionStream().id()));
assertTrue(distributor.isChild(streamE.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(2, distributor.numChildren(streamC.id()));
setPriority(streamF.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT, false);
assertEquals(1, distributor.numChildren(connection.connectionStream().id()));
assertTrue(distributor.isChild(streamF.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(1, distributor.numChildren(streamD.id()));
assertEquals(6, connection.numActiveStreams());
// Exclusive re-prioritization of a->d.
setPriority(streamA.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT, true);
// Level 0
assertEquals(1, distributor.numChildren(connection.connectionStream().id()));
// Level 1
assertTrue(distributor.isChild(streamD.id(), connection.connectionStream().id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(1, distributor.numChildren(streamD.id()));
// Level 2
assertTrue(distributor.isChild(streamA.id(), streamD.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(3, distributor.numChildren(streamA.id()));
// Level 3
assertTrue(distributor.isChild(streamB.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamB.id()));
assertTrue(distributor.isChild(streamF.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamF.id()));
assertTrue(distributor.isChild(streamC.id(), streamA.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(1, distributor.numChildren(streamC.id()));
// Level 4;
assertTrue(distributor.isChild(streamE.id(), streamC.id(), DEFAULT_PRIORITY_WEIGHT));
assertEquals(0, distributor.numChildren(streamE.id()));
}
}

View File

@ -14,15 +14,10 @@
*/
package io.netty.handler.codec.http2;
import io.netty.util.collection.IntObjectHashMap;
import io.netty.util.collection.IntObjectMap;
import org.junit.Before;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.mockito.Mock;
import org.mockito.MockitoAnnotations;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.mockito.verification.VerificationMode;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT;
@ -44,7 +39,7 @@ import static org.mockito.Mockito.reset;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
public class WeightedFairQueueByteDistributorTest {
public class WeightedFairQueueByteDistributorTest extends AbstractWeightedFairQueueByteDistributorDependencyTest {
private static final int STREAM_A = 1;
private static final int STREAM_B = 3;
private static final int STREAM_C = 5;
@ -52,12 +47,6 @@ public class WeightedFairQueueByteDistributorTest {
private static final int STREAM_E = 9;
private static final int ALLOCATION_QUANTUM = 100;
private Http2Connection connection;
private WeightedFairQueueByteDistributor distributor;
@Mock
private StreamByteDistributor.Writer writer;
@Before
public void setup() throws Http2Exception {
MockitoAnnotations.initMocks(this);
@ -73,22 +62,8 @@ public class WeightedFairQueueByteDistributorTest {
connection.local().createStream(STREAM_B, false);
Http2Stream streamC = connection.local().createStream(STREAM_C, false);
Http2Stream streamD = connection.local().createStream(STREAM_D, false);
streamC.setPriority(STREAM_A, DEFAULT_PRIORITY_WEIGHT, false);
streamD.setPriority(STREAM_A, DEFAULT_PRIORITY_WEIGHT, false);
}
private Answer<Void> writeAnswer(final boolean closeIfNoFrame) {
return new Answer<Void>() {
@Override
public Void answer(InvocationOnMock in) throws Throwable {
Http2Stream stream = in.getArgumentAt(0, Http2Stream.class);
int numBytes = in.getArgumentAt(1, Integer.class);
int streamableBytes = distributor.streamableBytes0(stream) - numBytes;
boolean hasFrame = streamableBytes > 0;
updateStream(stream.id(), streamableBytes, hasFrame, hasFrame, closeIfNoFrame);
return null;
}
};
setPriority(streamC.id(), STREAM_A, DEFAULT_PRIORITY_WEIGHT, false);
setPriority(streamD.id(), STREAM_A, DEFAULT_PRIORITY_WEIGHT, false);
}
/**
@ -120,8 +95,8 @@ public class WeightedFairQueueByteDistributorTest {
updateStream(STREAM_C, 600, true);
updateStream(STREAM_D, 700, true);
stream(STREAM_B).setPriority(STREAM_A, DEFAULT_PRIORITY_WEIGHT, true);
stream(STREAM_D).setPriority(STREAM_C, DEFAULT_PRIORITY_WEIGHT, true);
setPriority(STREAM_B, STREAM_A, DEFAULT_PRIORITY_WEIGHT, true);
setPriority(STREAM_D, STREAM_C, DEFAULT_PRIORITY_WEIGHT, true);
// Block B, but it should still remain in the queue/tree structure.
updateStream(STREAM_B, 0, false);
@ -259,7 +234,7 @@ public class WeightedFairQueueByteDistributorTest {
updateStream(STREAM_C, 0, true);
updateStream(STREAM_D, 10, true);
stream(STREAM_B).setPriority(STREAM_A, DEFAULT_PRIORITY_WEIGHT, true);
setPriority(STREAM_B, STREAM_A, DEFAULT_PRIORITY_WEIGHT, true);
assertFalse(write(10));
verifyWrite(STREAM_A, 0);
@ -680,7 +655,7 @@ public class WeightedFairQueueByteDistributorTest {
updateStream(STREAM_C, 0, true);
updateStream(STREAM_D, 0, true);
stream(STREAM_B).setPriority(STREAM_A, DEFAULT_PRIORITY_WEIGHT, true);
setPriority(STREAM_B, STREAM_A, DEFAULT_PRIORITY_WEIGHT, true);
assertFalse(write(0));
verifyNeverWrite(STREAM_A);
@ -723,7 +698,7 @@ public class WeightedFairQueueByteDistributorTest {
updateStream(STREAM_C, 0, true);
updateStream(STREAM_D, 0, true);
stream(STREAM_B).setPriority(STREAM_A, DEFAULT_PRIORITY_WEIGHT, true);
setPriority(STREAM_B, STREAM_A, DEFAULT_PRIORITY_WEIGHT, true);
assertFalse(write(100));
verifyNeverWrite(STREAM_A);
@ -765,7 +740,7 @@ public class WeightedFairQueueByteDistributorTest {
updateStream(STREAM_C, 600, true);
updateStream(STREAM_D, 700, true);
stream(STREAM_B).setPriority(STREAM_A, DEFAULT_PRIORITY_WEIGHT, true);
setPriority(STREAM_B, STREAM_A, DEFAULT_PRIORITY_WEIGHT, true);
assertTrue(write(500));
assertEquals(400, captureWrites(STREAM_A));
@ -812,16 +787,9 @@ public class WeightedFairQueueByteDistributorTest {
@Test
public void bytesDistributedWithAdditionShouldBeCorrect() throws Http2Exception {
Http2Stream streamE = connection.local().createStream(STREAM_E, false);
streamE.setPriority(STREAM_A, DEFAULT_PRIORITY_WEIGHT, true);
setPriority(streamE.id(), STREAM_A, DEFAULT_PRIORITY_WEIGHT, true);
// Send a bunch of data on each stream.
final IntObjectMap<Integer> streamSizes = new IntObjectHashMap<Integer>(4);
streamSizes.put(STREAM_A, (Integer) 400);
streamSizes.put(STREAM_B, (Integer) 500);
streamSizes.put(STREAM_C, (Integer) 600);
streamSizes.put(STREAM_D, (Integer) 700);
streamSizes.put(STREAM_E, (Integer) 900);
updateStream(STREAM_A, 400, true);
updateStream(STREAM_B, 500, true);
updateStream(STREAM_C, 600, true);
@ -954,14 +922,6 @@ public class WeightedFairQueueByteDistributorTest {
verify(writer, never()).write(same(stream), anyInt());
}
private void setPriority(int streamId, int parent, int weight, boolean exclusive) throws Http2Exception {
stream(streamId).setPriority(parent, (short) weight, exclusive);
}
private Http2Stream stream(int streamId) {
return connection.stream(streamId);
}
private int captureWrites(int streamId) {
return captureWrites(stream(streamId));
}
@ -975,37 +935,4 @@ public class WeightedFairQueueByteDistributorTest {
}
return total;
}
private void updateStream(final int streamId, final int streamableBytes, final boolean hasFrame) {
updateStream(streamId, streamableBytes, hasFrame, hasFrame, false);
}
private void updateStream(final int streamId, final int pendingBytes, final boolean hasFrame,
final boolean isWriteAllowed, boolean closeIfNoFrame) {
final Http2Stream stream = stream(streamId);
if (closeIfNoFrame && !hasFrame) {
stream(streamId).close();
}
distributor.updateStreamableBytes(new StreamByteDistributor.StreamState() {
@Override
public Http2Stream stream() {
return stream;
}
@Override
public int pendingBytes() {
return pendingBytes;
}
@Override
public boolean hasFrame() {
return hasFrame;
}
@Override
public int windowSize() {
return isWriteAllowed ? pendingBytes : -1;
}
});
}
}

View File

@ -0,0 +1,291 @@
/*
* Copyright 2015 The Netty Project
*
* The Netty Project 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 io.netty.util.internal;
import java.util.AbstractQueue;
import java.util.Arrays;
import java.util.Comparator;
import java.util.Iterator;
import java.util.NoSuchElementException;
import static io.netty.util.internal.PriorityQueueNode.INDEX_NOT_IN_QUEUE;
/**
* A priority queue which uses natural ordering of elements. Elements are also required to be of type
* {@link PriorityQueueNode} for the purpose of maintaining the index in the priority queue.
* @param <T> The object that is maintained in the queue.
*/
public final class DefaultPriorityQueue<T extends PriorityQueueNode> extends AbstractQueue<T>
implements PriorityQueue<T> {
private static final PriorityQueueNode[] EMPTY_ARRAY = new PriorityQueueNode[0];
private final Comparator<T> comparator;
private T[] queue;
private int size;
@SuppressWarnings("unchecked")
public DefaultPriorityQueue(Comparator<T> comparator, int initialSize) {
this.comparator = ObjectUtil.checkNotNull(comparator, "comparator");
queue = (T[]) (initialSize != 0 ? new PriorityQueueNode[initialSize] : EMPTY_ARRAY);
}
@Override
public int size() {
return size;
}
@Override
public boolean isEmpty() {
return size == 0;
}
@Override
public boolean contains(Object o) {
if (!(o instanceof PriorityQueueNode)) {
return false;
}
PriorityQueueNode node = (PriorityQueueNode) o;
return contains(node, node.priorityQueueIndex(this));
}
@Override
public boolean containsTyped(T node) {
return contains(node, node.priorityQueueIndex(this));
}
@Override
public void clear() {
for (int i = 0; i < size; ++i) {
T node = queue[i];
if (node != null) {
node.priorityQueueIndex(this, INDEX_NOT_IN_QUEUE);
queue[i] = null;
}
}
size = 0;
}
@Override
public boolean offer(T e) {
if (e.priorityQueueIndex(this) != INDEX_NOT_IN_QUEUE) {
throw new IllegalArgumentException("e.priorityQueueIndex(): " + e.priorityQueueIndex(this) +
" (expected: " + INDEX_NOT_IN_QUEUE + ") + e: " + e);
}
// Check that the array capacity is enough to hold values by doubling capacity.
if (size >= queue.length) {
// Use a policy which allows for a 0 initial capacity. Same policy as JDK's priority queue, double when
// "small", then grow by 50% when "large".
queue = Arrays.copyOf(queue, queue.length + ((queue.length < 64) ?
(queue.length + 2) :
(queue.length >>> 1)));
}
bubbleUp(size++, e);
return true;
}
@Override
public T poll() {
if (size == 0) {
return null;
}
T result = queue[0];
result.priorityQueueIndex(this, INDEX_NOT_IN_QUEUE);
T last = queue[--size];
queue[size] = null;
if (size != 0) { // Make sure we don't add the last element back.
bubbleDown(0, last);
}
return result;
}
@Override
public T peek() {
return (size == 0) ? null : queue[0];
}
@SuppressWarnings("unchecked")
@Override
public boolean remove(Object o) {
final T node;
try {
node = (T) o;
} catch (ClassCastException e) {
return false;
}
return removeTyped(node);
}
@Override
public boolean removeTyped(T node) {
int i = node.priorityQueueIndex(this);
if (!contains(node, i)) {
return false;
}
node.priorityQueueIndex(this, INDEX_NOT_IN_QUEUE);
if (--size == 0 || size == i) {
// If there are no node left, or this is the last node in the array just remove and return.
queue[i] = null;
return true;
}
// Move the last element where node currently lives in the array.
T moved = queue[i] = queue[size];
queue[size] = null;
// priorityQueueIndex will be updated below in bubbleUp or bubbleDown
// Make sure the moved node still preserves the min-heap properties.
if (comparator.compare(node, moved) < 0) {
bubbleDown(i, moved);
} else {
bubbleUp(i, moved);
}
return true;
}
@Override
public void priorityChanged(T node) {
int i = node.priorityQueueIndex(this);
if (!contains(node, i)) {
return;
}
// Preserve the min-heap property by comparing the new priority with parents/children in the heap.
if (i == 0) {
bubbleDown(i, node);
} else {
// Get the parent to see if min-heap properties are violated.
int iParent = (i - 1) >>> 1;
T parent = queue[iParent];
if (comparator.compare(node, parent) < 0) {
bubbleUp(i, node);
} else {
bubbleDown(i, node);
}
}
}
@Override
public Object[] toArray() {
return Arrays.copyOf(queue, size);
}
@SuppressWarnings("unchecked")
@Override
public <X> X[] toArray(X[] a) {
if (a.length < size) {
return (X[]) Arrays.copyOf(queue, size, a.getClass());
}
System.arraycopy(queue, 0, a, 0, size);
if (a.length > size) {
a[size] = null;
}
return a;
}
/**
* This iterator does not return elements in any particular order.
*/
@Override
public Iterator<T> iterator() {
return new PriorityQueueIterator();
}
private final class PriorityQueueIterator implements Iterator<T> {
private int index;
@Override
public boolean hasNext() {
return index < size;
}
@Override
public T next() {
if (index >= size) {
throw new NoSuchElementException();
}
return queue[index++];
}
@Override
public void remove() {
throw new UnsupportedOperationException("remove");
}
}
private boolean contains(PriorityQueueNode node, int i) {
return i >= 0 && i < size && node.equals(queue[i]);
}
private void bubbleDown(int k, T node) {
final int half = size >>> 1;
while (k < half) {
// Compare node to the children of index k.
int iChild = (k << 1) + 1;
T child = queue[iChild];
// Make sure we get the smallest child to compare against.
int rightChild = iChild + 1;
if (rightChild < size && comparator.compare(child, queue[rightChild]) > 0) {
child = queue[iChild = rightChild];
}
// If the bubbleDown node is less than or equal to the smallest child then we will preserve the min-heap
// property by inserting the bubbleDown node here.
if (comparator.compare(node, child) <= 0) {
break;
}
// Bubble the child up.
queue[k] = child;
child.priorityQueueIndex(this, k);
// Move down k down the tree for the next iteration.
k = iChild;
}
// We have found where node should live and still satisfy the min-heap property, so put it in the queue.
queue[k] = node;
node.priorityQueueIndex(this, k);
}
private void bubbleUp(int k, T node) {
while (k > 0) {
int iParent = (k - 1) >>> 1;
T parent = queue[iParent];
// If the bubbleUp node is less than the parent, then we have found a spot to insert and still maintain
// min-heap properties.
if (comparator.compare(node, parent) >= 0) {
break;
}
// Bubble the parent down.
queue[k] = parent;
parent.priorityQueueIndex(this, k);
// Move k up the tree for the next iteration.
k = iParent;
}
// We have found where node should live and still satisfy the min-heap property, so put it in the queue.
queue[k] = node;
node.priorityQueueIndex(this, k);
}
}

View File

@ -0,0 +1,157 @@
/*
* Copyright 2017 The Netty Project
*
* The Netty Project 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 io.netty.util.internal;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.NoSuchElementException;
public final class EmptyPriorityQueue<T> implements PriorityQueue<T> {
private static final PriorityQueue<Object> INSTANCE = new EmptyPriorityQueue<Object>();
private EmptyPriorityQueue() {
}
/**
* Returns an unmodifiable empty {@link PriorityQueue}.
*/
@SuppressWarnings("unchecked")
public static <V> EmptyPriorityQueue<V> instance() {
return (EmptyPriorityQueue<V>) INSTANCE;
}
@Override
public boolean removeTyped(T node) {
return false;
}
@Override
public boolean containsTyped(T node) {
return false;
}
@Override
public void priorityChanged(T node) {
}
@Override
public int size() {
return 0;
}
@Override
public boolean isEmpty() {
return true;
}
@Override
public boolean contains(Object o) {
return false;
}
@Override
public Iterator<T> iterator() {
return Collections.<T>emptyList().iterator();
}
@Override
public Object[] toArray() {
return EmptyArrays.EMPTY_OBJECTS;
}
@Override
public <T1> T1[] toArray(T1[] a) {
if (a.length > 0) {
a[0] = null;
}
return a;
}
@Override
public boolean add(T t) {
return false;
}
@Override
public boolean remove(Object o) {
return false;
}
@Override
public boolean containsAll(Collection<?> c) {
return false;
}
@Override
public boolean addAll(Collection<? extends T> c) {
return false;
}
@Override
public boolean removeAll(Collection<?> c) {
return false;
}
@Override
public boolean retainAll(Collection<?> c) {
return false;
}
@Override
public void clear() {
}
@Override
public boolean equals(Object o) {
return o instanceof PriorityQueue && ((PriorityQueue) o).isEmpty();
}
@Override
public int hashCode() {
return 0;
}
@Override
public boolean offer(T t) {
return false;
}
@Override
public T remove() {
throw new NoSuchElementException();
}
@Override
public T poll() {
return null;
}
@Override
public T element() {
throw new NoSuchElementException();
}
@Override
public T peek() {
return null;
}
@Override
public String toString() {
return EmptyPriorityQueue.class.getSimpleName();
}
}

View File

@ -65,7 +65,7 @@ public final class MathUtil {
}
/**
* Compare to {@code long} values.
* Compare two {@code long} values.
* @param x the first {@code long} to compare.
* @param y the second {@code long} to compare.
* @return

View File

@ -1,5 +1,5 @@
/*
* Copyright 2015 The Netty Project
* Copyright 2017 The Netty Project
*
* The Netty Project licenses this file to you under the Apache License,
* version 2.0 (the "License"); you may not use this file except in compliance
@ -15,240 +15,23 @@
*/
package io.netty.util.internal;
import java.util.AbstractQueue;
import java.util.Arrays;
import java.util.Iterator;
import java.util.NoSuchElementException;
import java.util.Queue;
import static io.netty.util.internal.ObjectUtil.checkNotNull;
import static io.netty.util.internal.PriorityQueueNode.INDEX_NOT_IN_QUEUE;
/**
* A priority queue which uses natural ordering of elements. Elements are also required to be of type
* {@link PriorityQueueNode} for the purpose of maintaining the index in the priority queue.
* @param <T> The object that is maintained in the queue.
*/
public final class PriorityQueue<T extends PriorityQueueNode<T>> extends AbstractQueue<T> implements Queue<T> {
@SuppressWarnings("rawtypes")
private static final PriorityQueueNode[] EMPTY_QUEUE = new PriorityQueueNode[0];
private T[] queue;
private int size;
public PriorityQueue() {
this(8);
}
@SuppressWarnings("unchecked")
public PriorityQueue(int initialSize) {
queue = (T[]) (initialSize != 0 ? new PriorityQueueNode[initialSize] : EMPTY_QUEUE);
}
@Override
public int size() {
return size;
}
@Override
public boolean isEmpty() {
return size == 0;
}
@Override
public boolean contains(Object o) {
if (!(o instanceof PriorityQueueNode)) {
return false;
}
PriorityQueueNode<?> node = (PriorityQueueNode<?>) o;
int i = node.priorityQueueIndex();
return i >= 0 && i < size && node.equals(queue[i]);
}
@Override
public void clear() {
for (int i = 0; i < size; ++i) {
T node = queue[i];
if (node != null) {
node.priorityQueueIndex(INDEX_NOT_IN_QUEUE);
queue[i] = null;
}
}
size = 0;
}
@Override
public boolean offer(T e) {
if (e.priorityQueueIndex() != INDEX_NOT_IN_QUEUE) {
throw new IllegalArgumentException("e.priorityQueueIndex(): " + e.priorityQueueIndex() +
" (expected: " + INDEX_NOT_IN_QUEUE + ") + e: " + e);
}
// Check that the array capacity is enough to hold values by doubling capacity.
if (size >= queue.length) {
// Use a policy which allows for a 0 initial capacity. Same policy as JDK's priority queue, double when
// "small", then grow by 50% when "large".
queue = Arrays.copyOf(queue, queue.length + ((queue.length < 64) ?
(queue.length + 2) :
(queue.length >>> 1)));
}
bubbleUp(size++, e);
return true;
}
@Override
public T poll() {
if (size == 0) {
return null;
}
T result = queue[0];
result.priorityQueueIndex(INDEX_NOT_IN_QUEUE);
T last = queue[--size];
queue[size] = null;
if (size != 0) { // Make sure we don't add the last element back.
bubbleDown(0, last);
}
return result;
}
@Override
public T peek() {
return (size == 0) ? null : queue[0];
}
@Override
public boolean remove(Object o) {
if (!contains(o)) {
return false;
}
@SuppressWarnings("unchecked")
T node = (T) o;
int i = node.priorityQueueIndex();
node.priorityQueueIndex(INDEX_NOT_IN_QUEUE);
if (--size == 0 || size == i) {
// If there are no node left, or this is the last node in the array just remove and return.
queue[i] = null;
return true;
}
// Move the last element where node currently lives in the array.
T moved = queue[i] = queue[size];
queue[size] = null;
// priorityQueueIndex will be updated below in bubbleUp or bubbleDown
// Make sure the moved node still preserves the min-heap properties.
if (node.compareTo(moved) < 0) {
bubbleDown(i, moved);
} else {
bubbleUp(i, moved);
}
return true;
}
@Override
public Object[] toArray() {
return Arrays.copyOf(queue, size);
}
@SuppressWarnings("unchecked")
@Override
public <X> X[] toArray(X[] a) {
if (a.length < size) {
return (X[]) Arrays.copyOf(queue, size, a.getClass());
}
System.arraycopy(queue, 0, a, 0, size);
if (a.length > size) {
a[size] = null;
}
return a;
}
public interface PriorityQueue<T> extends Queue<T> {
/**
* Same as {@link #remove(Object)} but typed using generics.
*/
boolean removeTyped(T node);
/**
* This iterator does not return elements in any particular order.
* Same as {@link #contains(Object)} but typed using generics.
*/
@Override
public Iterator<T> iterator() {
return new PriorityQueueIterator();
}
boolean containsTyped(T node);
private final class PriorityQueueIterator implements Iterator<T> {
private int index;
@Override
public boolean hasNext() {
return index < size;
}
@Override
public T next() {
if (index >= size) {
throw new NoSuchElementException();
}
return queue[index++];
}
@Override
public void remove() {
throw new UnsupportedOperationException("remove");
}
}
private void bubbleDown(int k, T node) {
final int half = size >>> 1;
while (k < half) {
// Compare node to the children of index k.
int iChild = (k << 1) + 1;
T child = queue[iChild];
// Make sure we get the smallest child to compare against.
int rightChild = iChild + 1;
if (rightChild < size && child.compareTo(queue[rightChild]) > 0) {
child = queue[iChild = rightChild];
}
// If the bubbleDown node is less than or equal to the smallest child then we will preserve the min-heap
// property by inserting the bubbleDown node here.
if (node.compareTo(child) <= 0) {
break;
}
// Bubble the child up.
queue[k] = child;
child.priorityQueueIndex(k);
// Move down k down the tree for the next iteration.
k = iChild;
}
// We have found where node should live and still satisfy the min-heap property, so put it in the queue.
queue[k] = node;
node.priorityQueueIndex(k);
}
private void bubbleUp(int k, T node) {
while (k > 0) {
int iParent = (k - 1) >>> 1;
T parent = queue[iParent];
// If the bubbleUp node is less than the parent, then we have found a spot to insert and still maintain
// min-heap properties.
if (node.compareTo(parent) >= 0) {
break;
}
// Bubble the parent down.
queue[k] = parent;
parent.priorityQueueIndex(k);
// Move k up the tree for the next iteration.
k = iParent;
}
// We have found where node should live and still satisfy the min-heap property, so put it in the queue.
queue[k] = node;
node.priorityQueueIndex(k);
}
/**
* Notify the queue that the priority for {@code node} has changed. The queue will adjust to ensure the priority
* queue properties are maintained.
* @param node An object which is in this queue and the priority may have changed.
*/
void priorityChanged(T node);
}

View File

@ -16,28 +16,29 @@
package io.netty.util.internal;
/**
* Provides methods for {@link PriorityQueue} to maintain internal state. These methods should generally not be used
* outside the scope of {@link PriorityQueue}.
* @param <T> The type which will be queued in {@link PriorityQueue}.
* Provides methods for {@link DefaultPriorityQueue} to maintain internal state. These methods should generally not be
* used outside the scope of {@link DefaultPriorityQueue}.
*/
public interface PriorityQueueNode<T> extends Comparable<T> {
public interface PriorityQueueNode {
/**
* This should be used to initialize the storage returned by {@link #priorityQueueIndex()}.
* This should be used to initialize the storage returned by {@link #priorityQueueIndex(DefaultPriorityQueue)}.
*/
int INDEX_NOT_IN_QUEUE = -1;
/**
* Get the last value set by {@link #priorityQueueIndex(int)}.
* Get the last value set by {@link #priorityQueueIndex(DefaultPriorityQueue, int)} for the value corresponding to
* {@code queue}.
* <p>
* Throwing exceptions from this method will result in undefined behavior.
*/
int priorityQueueIndex();
int priorityQueueIndex(DefaultPriorityQueue<?> queue);
/**
* Used by {@link PriorityQueue} to maintain state for an element in the queue.
* Used by {@link DefaultPriorityQueue} to maintain state for an element in the queue.
* <p>
* Throwing exceptions from this method will result in undefined behavior.
* @param i The index as used by {@link PriorityQueue}.
* @param queue The queue for which the index is being set.
* @param i The index as used by {@link DefaultPriorityQueue}.
*/
void priorityQueueIndex(int i);
void priorityQueueIndex(DefaultPriorityQueue<?> queue, int i);
}

View File

@ -17,6 +17,13 @@ package io.netty.util.internal;
import org.junit.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
@ -24,10 +31,10 @@ import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class PriorityQueueTest {
public class DefaultPriorityQueueTest {
@Test
public void testPoll() {
PriorityQueue<TestElement> queue = new PriorityQueue<TestElement>(0);
PriorityQueue<TestElement> queue = new DefaultPriorityQueue<TestElement>(TestElementComparator.INSTANCE, 0);
assertEmptyQueue(queue);
TestElement a = new TestElement(5);
@ -69,7 +76,7 @@ public class PriorityQueueTest {
@Test
public void testClear() {
PriorityQueue<TestElement> queue = new PriorityQueue<TestElement>(0);
PriorityQueue<TestElement> queue = new DefaultPriorityQueue<TestElement>(TestElementComparator.INSTANCE, 0);
assertEmptyQueue(queue);
TestElement a = new TestElement(5);
@ -101,7 +108,16 @@ public class PriorityQueueTest {
@Test
public void testRemoval() {
PriorityQueue<TestElement> queue = new PriorityQueue<TestElement>(4);
testRemoval(false);
}
@Test
public void testRemovalTyped() {
testRemoval(true);
}
private void testRemoval(boolean typed) {
PriorityQueue<TestElement> queue = new DefaultPriorityQueue<TestElement>(TestElementComparator.INSTANCE, 4);
assertEmptyQueue(queue);
TestElement a = new TestElement(5);
@ -116,12 +132,12 @@ public class PriorityQueueTest {
assertOffer(queue, d);
// Remove an element that isn't in the queue.
assertFalse(queue.remove(notInQueue));
assertFalse(typed ? queue.removeTyped(notInQueue) : queue.remove(notInQueue));
assertSame(c, queue.peek());
assertEquals(4, queue.size());
// Remove the last element in the array, when the array is non-empty.
assertTrue(queue.remove(b));
assertTrue(typed ? queue.removeTyped(b) : queue.remove(b));
assertSame(c, queue.peek());
assertEquals(3, queue.size());
@ -131,26 +147,26 @@ public class PriorityQueueTest {
assertEquals(4, queue.size());
// Repeat remove the last element in the array, when the array is non-empty.
assertTrue(queue.remove(b));
assertTrue(typed ? queue.removeTyped(b) : queue.remove(b));
assertSame(c, queue.peek());
assertEquals(3, queue.size());
// Remove the head of the queue.
assertTrue(queue.remove(c));
assertTrue(typed ? queue.removeTyped(c) : queue.remove(c));
assertSame(a, queue.peek());
assertEquals(2, queue.size());
assertTrue(queue.remove(a));
assertTrue(typed ? queue.removeTyped(a) : queue.remove(a));
assertSame(d, queue.peek());
assertEquals(1, queue.size());
assertTrue(queue.remove(d));
assertTrue(typed ? queue.removeTyped(d) : queue.remove(d));
assertEmptyQueue(queue);
}
@Test
public void testZeroInitialSize() {
PriorityQueue<TestElement> queue = new PriorityQueue<TestElement>(0);
PriorityQueue<TestElement> queue = new DefaultPriorityQueue<TestElement>(TestElementComparator.INSTANCE, 0);
assertEmptyQueue(queue);
TestElement e = new TestElement(1);
assertOffer(queue, e);
@ -161,9 +177,59 @@ public class PriorityQueueTest {
assertEmptyQueue(queue);
}
@Test
public void testPriorityChange() {
PriorityQueue<TestElement> queue = new DefaultPriorityQueue<TestElement>(TestElementComparator.INSTANCE, 0);
assertEmptyQueue(queue);
TestElement a = new TestElement(10);
TestElement b = new TestElement(20);
TestElement c = new TestElement(30);
TestElement d = new TestElement(25);
TestElement e = new TestElement(23);
TestElement f = new TestElement(15);
queue.add(a);
queue.add(b);
queue.add(c);
queue.add(d);
queue.add(e);
queue.add(f);
e.value = 35;
queue.priorityChanged(e);
a.value = 40;
queue.priorityChanged(a);
a.value = 31;
queue.priorityChanged(a);
d.value = 10;
queue.priorityChanged(d);
f.value = 5;
queue.priorityChanged(f);
List<TestElement> expectedOrderList = new ArrayList<TestElement>(queue.size());
expectedOrderList.addAll(Arrays.asList(a, b, c, d, e, f));
Collections.sort(expectedOrderList, TestElementComparator.INSTANCE);
assertEquals(expectedOrderList.size(), queue.size());
assertEquals(expectedOrderList.isEmpty(), queue.isEmpty());
Iterator<TestElement> itr = expectedOrderList.iterator();
while (itr.hasNext()) {
TestElement next = itr.next();
TestElement poll = queue.poll();
assertEquals(next, poll);
itr.remove();
assertEquals(expectedOrderList.size(), queue.size());
assertEquals(expectedOrderList.isEmpty(), queue.isEmpty());
}
}
private static void assertOffer(PriorityQueue<TestElement> queue, TestElement a) {
assertTrue(queue.offer(a));
assertTrue(queue.contains(a));
assertTrue(queue.containsTyped(a));
try { // An element can not be inserted more than 1 time.
queue.offer(a);
fail();
@ -179,25 +245,29 @@ public class PriorityQueueTest {
assertTrue(queue.isEmpty());
}
private static final class TestElement implements Comparable<TestElement>, PriorityQueueNode<TestElement> {
private static final class TestElementComparator implements Comparator<TestElement> {
static final TestElementComparator INSTANCE = new TestElementComparator();
private TestElementComparator() {
}
@Override
public int compare(TestElement o1, TestElement o2) {
return o1.value - o2.value;
}
}
private static final class TestElement implements PriorityQueueNode {
int value;
private int priorityQueueIndex = INDEX_NOT_IN_QUEUE;
public TestElement(int value) {
TestElement(int value) {
this.value = value;
}
@Override
public int compareTo(TestElement o) {
return value - o.value;
}
@Override
public boolean equals(Object o) {
if (!(o instanceof TestElement)) {
return false;
}
return ((TestElement) o).value == value;
return o instanceof TestElement && ((TestElement) o).value == value;
}
@Override
@ -206,12 +276,12 @@ public class PriorityQueueTest {
}
@Override
public int priorityQueueIndex() {
public int priorityQueueIndex(DefaultPriorityQueue queue) {
return priorityQueueIndex;
}
@Override
public void priorityQueueIndex(int i) {
public void priorityQueueIndex(DefaultPriorityQueue queue, int i) {
priorityQueueIndex = i;
}
}

View File

@ -1,109 +0,0 @@
/*
* Copyright 2015 The Netty Project
*
* The Netty Project 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 io.netty.microbench.http2;
import io.netty.handler.codec.http2.DefaultHttp2Connection;
import io.netty.handler.codec.http2.Http2Connection;
import io.netty.handler.codec.http2.Http2Exception;
import io.netty.handler.codec.http2.Http2Stream;
import io.netty.microbench.util.AbstractMicrobenchmark;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.Level;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.TearDown;
import org.openjdk.jmh.annotations.Threads;
/**
* This benchmark is designed to exercise re-prioritization events and measure the ops/sec.
*/
@Threads(1)
@State(Scope.Benchmark)
public class Http2PriorityTreeBenchmark extends AbstractMicrobenchmark {
private static final short WEIGHT = 1;
public Http2Connection connection;
@Param({ "1000", "10000" })
public int numStreams;
@Param({ "10", "100", "1000" })
public int d_ary;
@Param({ "10", "100", "1000" })
public int exclusiveOnCount;
@Setup(Level.Trial)
public void setup() throws Http2Exception {
connection = new DefaultHttp2Connection(false);
for (int i = 0; i < numStreams; ++i) {
connection.local().createStream(toStreamId(i), false);
}
}
@TearDown(Level.Iteration)
public void teardown() throws Http2Exception {
final int connectionId = connection.connectionStream().id();
for (int i = 0; i < numStreams; ++i) {
connection.stream(toStreamId(i)).setPriority(connectionId, WEIGHT, false);
}
}
/**
* A priority tree will be build using the {@link #d_ary} variable to determine the number of children for each
* node. After the priority tree is built the nodes closest to the root will be pushed down to be dependent on leaf
* nodes. This is to simulate an "expensive" tree operation.
*/
@Benchmark
public void prioritizeStreams() throws Http2Exception {
int streamId = 0;
int parentId = 0;
boolean exclusive = false;
for (int i = 0; i < numStreams; ++i) {
// Treat all streams as they exist in a logical array in the range [0, numStreams].
// From this we can find the current parent stream via a i / d_ary operation.
parentId = toStreamId(i / d_ary);
streamId = toStreamId(i);
if (parentId == streamId) {
exclusive = i % exclusiveOnCount == 0;
continue;
}
Http2Stream stream = connection.stream(streamId);
stream.setPriority(parentId, WEIGHT, exclusive);
exclusive = i % exclusiveOnCount == 0;
}
// Now change the parent assignments by pushing the root nodes out to the leafs.
for (int i = 0; i < numStreams; ++i) {
parentId = toStreamId((numStreams - i) / d_ary);
streamId = toStreamId(i);
if (parentId == streamId) {
exclusive = i % exclusiveOnCount == 0;
continue;
}
Http2Stream stream = connection.stream(streamId);
stream.setPriority(parentId, WEIGHT, exclusive);
exclusive = i % exclusiveOnCount == 0;
}
}
private static int toStreamId(int i) {
return 2 * i + 1;
}
}

View File

@ -31,7 +31,6 @@ import io.netty.handler.codec.http2.UniformStreamByteDistributor;
import io.netty.handler.codec.http2.WeightedFairQueueByteDistributor;
import io.netty.microbench.channel.EmbeddedChannelWriteReleaseHandlerContext;
import io.netty.microbench.util.AbstractMicrobenchmark;
import org.openjdk.jmh.annotations.AuxCounters;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.Level;
@ -80,8 +79,8 @@ public class NoPriorityByteDistributionBenchmark extends AbstractMicrobenchmark
@AuxCounters
@State(Scope.Thread)
public static class AdditionalCounters {
int minWriteSize;
int maxWriteSize;
int minWriteSize = Integer.MAX_VALUE;
int maxWriteSize = Integer.MIN_VALUE;
long totalBytes;
long numWrites;
int invocations;
@ -124,7 +123,7 @@ public class NoPriorityByteDistributionBenchmark extends AbstractMicrobenchmark
// Create the flow controller
switch (algorithm) {
case WFQ:
distributor = new WeightedFairQueueByteDistributor(connection);
distributor = new WeightedFairQueueByteDistributor(connection, 0);
break;
case UNIFORM:
distributor = new UniformStreamByteDistributor(connection);
@ -137,8 +136,7 @@ public class NoPriorityByteDistributionBenchmark extends AbstractMicrobenchmark
.frameListener(new Http2FrameAdapter())
.connection(connection)
.build();
ctx = new EmbeddedChannelWriteReleaseHandlerContext(
PooledByteBufAllocator.DEFAULT, handler) {
ctx = new EmbeddedChannelWriteReleaseHandlerContext(PooledByteBufAllocator.DEFAULT, handler) {
@Override
protected void handleException(Throwable t) {
handleUnexpectedException(t);
@ -182,7 +180,7 @@ public class NoPriorityByteDistributionBenchmark extends AbstractMicrobenchmark
private void addData(Http2Stream stream, final int dataSize) {
controller.addFlowControlled(stream, new Http2RemoteFlowController.FlowControlled() {
int size = dataSize;
private int size = dataSize;
@Override
public int size() {
@ -257,8 +255,12 @@ public class NoPriorityByteDistributionBenchmark extends AbstractMicrobenchmark
}
@Override
public boolean distribute(int maxBytes, Writer writer)
throws Http2Exception {
public void updateDependencyTree(int childStreamId, int parentStreamId, short weight, boolean exclusive) {
delegate.updateDependencyTree(childStreamId, parentStreamId, weight, exclusive);
}
@Override
public boolean distribute(int maxBytes, Writer writer) throws Http2Exception {
return delegate.distribute(maxBytes, new CountingWriter(writer));
}
@ -277,9 +279,9 @@ public class NoPriorityByteDistributionBenchmark extends AbstractMicrobenchmark
DataRefresher refresher = dataRefresher(stream);
refresher.add(numBytes);
counters.numWrites++;
++counters.numWrites;
counters.totalBytes += numBytes;
if (counters.minWriteSize == 0 || numBytes < counters.minWriteSize) {
if (numBytes < counters.minWriteSize) {
counters.minWriteSize = numBytes;
}
if (numBytes > counters.maxWriteSize) {

View File

@ -84,4 +84,8 @@ public final class NoopHttp2RemoteFlowController implements Http2RemoteFlowContr
@Override
public void channelWritabilityChanged() throws Http2Exception {
}
@Override
public void updateDependencyTree(int childStreamId, int parentStreamId, short weight, boolean exclusive) {
}
}