normanmaurer commented on code in PR #5350: URL: https://github.com/apache/hbase/pull/5350#discussion_r1297016147
########## hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerChannelWritabilityHandler.java: ########## @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import java.util.function.BooleanSupplier; +import java.util.function.IntSupplier; +import org.apache.hadoop.hbase.exceptions.ConnectionClosedException; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.NettyUnsafeUtils; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hbase.thirdparty.io.netty.channel.Channel; +import org.apache.hbase.thirdparty.io.netty.channel.ChannelDuplexHandler; +import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext; +import org.apache.hbase.thirdparty.io.netty.channel.ChannelPromise; + +/** + * Handler to enforce writability protections on our server channels: <br> + * - Responds to channel writability events, which are triggered when the total pending bytes for a + * channel passes configured high and low watermarks. When high watermark is exceeded, the channel + * is setAutoRead(false). This way, we won't accept new requests from the client until some pending + * outbound bytes are successfully received by the client.<br> + * - Pre-processes any channel write requests. If the total pending outbound bytes exceeds a fatal + * threshold, the channel is forcefully closed and the write is set to failed. This handler should + * be the last handler in the pipeline so that it's the first handler to receive any messages sent + * to channel.write() or channel.writeAndFlush(). + */ [email protected] +public class NettyRpcServerChannelWritabilityHandler extends ChannelDuplexHandler { + + private static final ConnectionClosedException EXCEPTION = + new ConnectionClosedException("Channel outbound bytes exceeded fatal threshold") { + @Override + public Throwable fillInStackTrace() { + // the stacktrace is noisy and not very useful here. + return this; + } + }; + + static final String NAME = "NettyRpcServerChannelWritabilityHandler"; + + private final MetricsHBaseServer metrics; + private final IntSupplier pendingBytesFatalThreshold; + private final BooleanSupplier isWritabilityBackpressureEnabled; + + private boolean writable = true; + private long unwritableStartTime; + + NettyRpcServerChannelWritabilityHandler(MetricsHBaseServer metrics, + IntSupplier pendingBytesFatalThreshold, BooleanSupplier isWritabilityBackpressureEnabled) { + this.metrics = metrics; + this.pendingBytesFatalThreshold = pendingBytesFatalThreshold; + this.isWritabilityBackpressureEnabled = isWritabilityBackpressureEnabled; + } + + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) + throws Exception { + if (handleFatalThreshold(ctx)) { + promise.setFailure(EXCEPTION); Review Comment: You must also release the msg via `ReferenceCountUtil.release(msg)` before return early as otherwise you might leak memory. ########## hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerChannelWritabilityHandler.java: ########## @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import java.util.function.BooleanSupplier; +import java.util.function.IntSupplier; +import org.apache.hadoop.hbase.exceptions.ConnectionClosedException; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.NettyUnsafeUtils; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hbase.thirdparty.io.netty.channel.Channel; +import org.apache.hbase.thirdparty.io.netty.channel.ChannelDuplexHandler; +import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext; +import org.apache.hbase.thirdparty.io.netty.channel.ChannelPromise; + +/** + * Handler to enforce writability protections on our server channels: <br> + * - Responds to channel writability events, which are triggered when the total pending bytes for a + * channel passes configured high and low watermarks. When high watermark is exceeded, the channel + * is setAutoRead(false). This way, we won't accept new requests from the client until some pending + * outbound bytes are successfully received by the client.<br> + * - Pre-processes any channel write requests. If the total pending outbound bytes exceeds a fatal + * threshold, the channel is forcefully closed and the write is set to failed. This handler should + * be the last handler in the pipeline so that it's the first handler to receive any messages sent + * to channel.write() or channel.writeAndFlush(). + */ [email protected] +public class NettyRpcServerChannelWritabilityHandler extends ChannelDuplexHandler { + + private static final ConnectionClosedException EXCEPTION = + new ConnectionClosedException("Channel outbound bytes exceeded fatal threshold") { + @Override + public Throwable fillInStackTrace() { + // the stacktrace is noisy and not very useful here. + return this; + } + }; + + static final String NAME = "NettyRpcServerChannelWritabilityHandler"; + + private final MetricsHBaseServer metrics; + private final IntSupplier pendingBytesFatalThreshold; + private final BooleanSupplier isWritabilityBackpressureEnabled; + + private boolean writable = true; + private long unwritableStartTime; + + NettyRpcServerChannelWritabilityHandler(MetricsHBaseServer metrics, + IntSupplier pendingBytesFatalThreshold, BooleanSupplier isWritabilityBackpressureEnabled) { + this.metrics = metrics; + this.pendingBytesFatalThreshold = pendingBytesFatalThreshold; + this.isWritabilityBackpressureEnabled = isWritabilityBackpressureEnabled; + } + + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) + throws Exception { + if (handleFatalThreshold(ctx)) { + promise.setFailure(EXCEPTION); Review Comment: Also as a side note reusing a static exception instance is only safe if the instance does not support `addSuppressed(...)` as otherwise it is possible that you will end up with a memory leak if someone add suppressed exceptions to the static instance, which is possible in this case as you dont know what listeners etc will do. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
