You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/03/16 09:12:00 UTC

[GitHub] [pulsar] lhotari opened a new pull request #14713: [Proxy] Refactor Proxy code

lhotari opened a new pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713


   ### Motivation
   
   Refactor Proxy code to make it easier to understand and maintain.
   
   
   ### Modifications
   
   - replace broker host parsing with a simple solution
   - pass remote host name to ProxyBackendHandler in the constructor
   - rename "targetBrokerUrl" to "brokerHostAndPort" since the "targetBrokerUrl" is really "hostname:port" string
   - move HA proxy message handling to ProxyBackendHandle and extract the logic to a method
   - remove the static "inboundOutboundChannelMap" which was used for log level 2 
     - make it obsolete by passing the peer channel id to ParserProxyHandler
    


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on a change in pull request #14713: [Proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#discussion_r829232363



##########
File path: pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java
##########
@@ -123,67 +128,52 @@ protected void initChannel(SocketChannel ch) {
                 }
                 ch.pipeline().addLast("frameDecoder", new LengthFieldBasedFrameDecoder(
                     Commands.DEFAULT_MAX_MESSAGE_SIZE + Commands.MESSAGE_SIZE_FRAME_PADDING, 0, 4, 0, 4));
-                ch.pipeline().addLast("proxyOutboundHandler", new ProxyBackendHandler(config, protocolVersion));
+                ch.pipeline().addLast("proxyOutboundHandler",
+                        new ProxyBackendHandler(config, protocolVersion, remoteHost));
             }
         });
 
-        URI targetBroker;
-        try {
-            // targetBrokerUrl is coming in the "hostname:6650" form, so we need
-            // to extract host and port
-            targetBroker = new URI("pulsar://" + targetBrokerUrl);
-        } catch (URISyntaxException e) {
-            log.warn("[{}] Failed to parse broker url '{}'", inboundChannel, targetBrokerUrl, e);
-            inboundChannel.close();
-            return;
-        }
-
         ChannelFuture f = b.connect(targetBrokerAddress);
         outboundChannel = f.channel();
         f.addListener(future -> {
             if (!future.isSuccess()) {
                 // Close the connection if the connection attempt has failed.
                 log.warn("[{}] Establishing connection to {} ({}) failed. Closing inbound channel.", inboundChannel,
-                        targetBrokerAddress, targetBrokerUrl, future.cause());
+                        targetBrokerAddress, brokerHostAndPort, future.cause());
                 inboundChannel.close();
                 return;
             }
-            final ProxyBackendHandler cnx = (ProxyBackendHandler) outboundChannel.pipeline()
-                    .get("proxyOutboundHandler");
-            cnx.setRemoteHostName(targetBroker.getHost());
-
-            // if enable full parsing feature
-            if (service.getProxyLogLevel() == 2) {
-                //Set a map between inbound and outbound,
-                //so can find inbound by outbound or find outbound by inbound
-                inboundOutboundChannelMap.put(outboundChannel.id(), inboundChannel.id());
-            }
+        });
+    }
 
-            if (!config.isHaProxyProtocolEnabled()) {
-                return;
-            }
+    private String parseHost(String brokerPortAndHost) {

Review comment:
       nit: static




-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on a change in pull request #14713: [Proxy] Refactor Proxy code

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#discussion_r828232254



##########
File path: pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java
##########
@@ -71,11 +66,11 @@
 
     @Getter
     private final Channel inboundChannel;
+    private final ProxyConnection proxyConnection;
     @Getter
     Channel outboundChannel;
     @Getter
     private final Rate inboundChannelRequestsRate;
-    protected static Map<ChannelId, ChannelId> inboundOutboundChannelMap = new ConcurrentHashMap<>();

Review comment:
       great to see this removed! 




-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui commented on a change in pull request #14713: [Proxy] Refactor Proxy code

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on a change in pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#discussion_r828915070



##########
File path: pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java
##########
@@ -71,11 +66,11 @@
 
     @Getter
     private final Channel inboundChannel;
+    private final ProxyConnection proxyConnection;
     @Getter
     Channel outboundChannel;
     @Getter
     private final Rate inboundChannelRequestsRate;
-    protected static Map<ChannelId, ChannelId> inboundOutboundChannelMap = new ConcurrentHashMap<>();

Review comment:
       Great remove.
   
   Look like the data in this map will never remove. 




-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on a change in pull request #14713: [Proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#discussion_r831319213



##########
File path: pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
##########
@@ -458,25 +459,6 @@ protected void handleLookup(CommandLookupTopic lookup) {
         lookupProxyHandler.handleLookup(lookup);
     }
 
-    private synchronized void close() {
-        if (state != State.Closed) {
-            state = State.Closed;
-            if (directProxyHandler != null && directProxyHandler.outboundChannel != null) {
-                directProxyHandler.outboundChannel.close();
-                directProxyHandler = null;
-            }
-            if (connectionPool != null) {
-                try {
-                    connectionPool.close();

Review comment:
       Yes, I removed it since there was duplication. the `channelInactive` method contains the cleanup.




-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on pull request #14713: [Proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode

Posted by GitBox <gi...@apache.org>.
lhotari commented on pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#issuecomment-1071102605


   @merlimat I have implemented the solution for flow control in a04aa5f . Please review
   
   


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] michaeljmarshall commented on a change in pull request #14713: [Proxy] Refactor Proxy code

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on a change in pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#discussion_r828191513



##########
File path: pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java
##########
@@ -123,65 +129,52 @@ protected void initChannel(SocketChannel ch) {
                 }
                 ch.pipeline().addLast("frameDecoder", new LengthFieldBasedFrameDecoder(
                     Commands.DEFAULT_MAX_MESSAGE_SIZE + Commands.MESSAGE_SIZE_FRAME_PADDING, 0, 4, 0, 4));
-                ch.pipeline().addLast("proxyOutboundHandler", new ProxyBackendHandler(config, protocolVersion));
+                ch.pipeline().addLast("proxyOutboundHandler",
+                        new ProxyBackendHandler(config, protocolVersion, remoteHost));
             }
         });
 
-        URI targetBroker;
-        try {
-            // targetBrokerUrl is coming in the "hostname:6650" form, so we need
-            // to extract host and port
-            targetBroker = new URI("pulsar://" + targetBrokerUrl);

Review comment:
       This is great, I was very confused by this line when reviewing one of your other PRs today.




-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on pull request #14713: [Proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode

Posted by GitBox <gi...@apache.org>.
lhotari commented on pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#issuecomment-1071027004






-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari edited a comment on pull request #14713: [Proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode

Posted by GitBox <gi...@apache.org>.
lhotari edited a comment on pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#issuecomment-1071027004






-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] michaeljmarshall commented on a change in pull request #14713: [Proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on a change in pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#discussion_r831284455



##########
File path: pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
##########
@@ -458,25 +459,6 @@ protected void handleLookup(CommandLookupTopic lookup) {
         lookupProxyHandler.handleLookup(lookup);
     }
 
-    private synchronized void close() {
-        if (state != State.Closed) {
-            state = State.Closed;
-            if (directProxyHandler != null && directProxyHandler.outboundChannel != null) {
-                directProxyHandler.outboundChannel.close();
-                directProxyHandler = null;
-            }
-            if (connectionPool != null) {
-                try {
-                    connectionPool.close();

Review comment:
       I was wondering if we needed to call close here, but looks like that'll happen in the `channelInactive` method that netty will call.




-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on a change in pull request #14713: [Proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#discussion_r829232363



##########
File path: pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java
##########
@@ -123,67 +128,52 @@ protected void initChannel(SocketChannel ch) {
                 }
                 ch.pipeline().addLast("frameDecoder", new LengthFieldBasedFrameDecoder(
                     Commands.DEFAULT_MAX_MESSAGE_SIZE + Commands.MESSAGE_SIZE_FRAME_PADDING, 0, 4, 0, 4));
-                ch.pipeline().addLast("proxyOutboundHandler", new ProxyBackendHandler(config, protocolVersion));
+                ch.pipeline().addLast("proxyOutboundHandler",
+                        new ProxyBackendHandler(config, protocolVersion, remoteHost));
             }
         });
 
-        URI targetBroker;
-        try {
-            // targetBrokerUrl is coming in the "hostname:6650" form, so we need
-            // to extract host and port
-            targetBroker = new URI("pulsar://" + targetBrokerUrl);
-        } catch (URISyntaxException e) {
-            log.warn("[{}] Failed to parse broker url '{}'", inboundChannel, targetBrokerUrl, e);
-            inboundChannel.close();
-            return;
-        }
-
         ChannelFuture f = b.connect(targetBrokerAddress);
         outboundChannel = f.channel();
         f.addListener(future -> {
             if (!future.isSuccess()) {
                 // Close the connection if the connection attempt has failed.
                 log.warn("[{}] Establishing connection to {} ({}) failed. Closing inbound channel.", inboundChannel,
-                        targetBrokerAddress, targetBrokerUrl, future.cause());
+                        targetBrokerAddress, brokerHostAndPort, future.cause());
                 inboundChannel.close();
                 return;
             }
-            final ProxyBackendHandler cnx = (ProxyBackendHandler) outboundChannel.pipeline()
-                    .get("proxyOutboundHandler");
-            cnx.setRemoteHostName(targetBroker.getHost());
-
-            // if enable full parsing feature
-            if (service.getProxyLogLevel() == 2) {
-                //Set a map between inbound and outbound,
-                //so can find inbound by outbound or find outbound by inbound
-                inboundOutboundChannelMap.put(outboundChannel.id(), inboundChannel.id());
-            }
+        });
+    }
 
-            if (!config.isHaProxyProtocolEnabled()) {
-                return;
-            }
+    private String parseHost(String brokerPortAndHost) {

Review comment:
       nit: static




-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on pull request #14713: [Proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode

Posted by GitBox <gi...@apache.org>.
lhotari commented on pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#issuecomment-1071027004


   I ended up increasing the scope of this PR and add the fix to a connection stalling issue. Please review again @eolivelli @codelipenghui @michaeljmarshall @nicoloboschi @merlimat 


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui merged pull request #14713: [Proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode

Posted by GitBox <gi...@apache.org>.
codelipenghui merged pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713


   


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on pull request #14713: [Proxy] Refactor Proxy code

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#issuecomment-1070571224


   @lhotari please resolve the conflicts


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on pull request #14713: [Proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode

Posted by GitBox <gi...@apache.org>.
lhotari commented on pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#issuecomment-1071114661


   The channel writability change can be configured with `setWriteBufferWaterMark` or `setWriteBufferLowWaterMark`/`setWriteBufferHighWaterMark` methods. By default, the low water mark is 32kB and high water mark is 64kB. That sounds like a sensible default.
   


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari edited a comment on pull request #14713: [Proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode

Posted by GitBox <gi...@apache.org>.
lhotari edited a comment on pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#issuecomment-1071027004


   I ended up increasing the scope of this PR and added the fix to a connection stalling issue. Please review again @eolivelli @codelipenghui @michaeljmarshall @nicoloboschi @merlimat 


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on a change in pull request #14713: [Proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#discussion_r829310927



##########
File path: pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java
##########
@@ -123,67 +128,52 @@ protected void initChannel(SocketChannel ch) {
                 }
                 ch.pipeline().addLast("frameDecoder", new LengthFieldBasedFrameDecoder(
                     Commands.DEFAULT_MAX_MESSAGE_SIZE + Commands.MESSAGE_SIZE_FRAME_PADDING, 0, 4, 0, 4));
-                ch.pipeline().addLast("proxyOutboundHandler", new ProxyBackendHandler(config, protocolVersion));
+                ch.pipeline().addLast("proxyOutboundHandler",
+                        new ProxyBackendHandler(config, protocolVersion, remoteHost));
             }
         });
 
-        URI targetBroker;
-        try {
-            // targetBrokerUrl is coming in the "hostname:6650" form, so we need
-            // to extract host and port
-            targetBroker = new URI("pulsar://" + targetBrokerUrl);
-        } catch (URISyntaxException e) {
-            log.warn("[{}] Failed to parse broker url '{}'", inboundChannel, targetBrokerUrl, e);
-            inboundChannel.close();
-            return;
-        }
-
         ChannelFuture f = b.connect(targetBrokerAddress);
         outboundChannel = f.channel();
         f.addListener(future -> {
             if (!future.isSuccess()) {
                 // Close the connection if the connection attempt has failed.
                 log.warn("[{}] Establishing connection to {} ({}) failed. Closing inbound channel.", inboundChannel,
-                        targetBrokerAddress, targetBrokerUrl, future.cause());
+                        targetBrokerAddress, brokerHostAndPort, future.cause());
                 inboundChannel.close();
                 return;
             }
-            final ProxyBackendHandler cnx = (ProxyBackendHandler) outboundChannel.pipeline()
-                    .get("proxyOutboundHandler");
-            cnx.setRemoteHostName(targetBroker.getHost());
-
-            // if enable full parsing feature
-            if (service.getProxyLogLevel() == 2) {
-                //Set a map between inbound and outbound,
-                //so can find inbound by outbound or find outbound by inbound
-                inboundOutboundChannelMap.put(outboundChannel.id(), inboundChannel.id());
-            }
+        });
+    }
 
-            if (!config.isHaProxyProtocolEnabled()) {
-                return;
-            }
+    private String parseHost(String brokerPortAndHost) {

Review comment:
       done




-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on a change in pull request #14713: [Proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#discussion_r829310927



##########
File path: pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java
##########
@@ -123,67 +128,52 @@ protected void initChannel(SocketChannel ch) {
                 }
                 ch.pipeline().addLast("frameDecoder", new LengthFieldBasedFrameDecoder(
                     Commands.DEFAULT_MAX_MESSAGE_SIZE + Commands.MESSAGE_SIZE_FRAME_PADDING, 0, 4, 0, 4));
-                ch.pipeline().addLast("proxyOutboundHandler", new ProxyBackendHandler(config, protocolVersion));
+                ch.pipeline().addLast("proxyOutboundHandler",
+                        new ProxyBackendHandler(config, protocolVersion, remoteHost));
             }
         });
 
-        URI targetBroker;
-        try {
-            // targetBrokerUrl is coming in the "hostname:6650" form, so we need
-            // to extract host and port
-            targetBroker = new URI("pulsar://" + targetBrokerUrl);
-        } catch (URISyntaxException e) {
-            log.warn("[{}] Failed to parse broker url '{}'", inboundChannel, targetBrokerUrl, e);
-            inboundChannel.close();
-            return;
-        }
-
         ChannelFuture f = b.connect(targetBrokerAddress);
         outboundChannel = f.channel();
         f.addListener(future -> {
             if (!future.isSuccess()) {
                 // Close the connection if the connection attempt has failed.
                 log.warn("[{}] Establishing connection to {} ({}) failed. Closing inbound channel.", inboundChannel,
-                        targetBrokerAddress, targetBrokerUrl, future.cause());
+                        targetBrokerAddress, brokerHostAndPort, future.cause());
                 inboundChannel.close();
                 return;
             }
-            final ProxyBackendHandler cnx = (ProxyBackendHandler) outboundChannel.pipeline()
-                    .get("proxyOutboundHandler");
-            cnx.setRemoteHostName(targetBroker.getHost());
-
-            // if enable full parsing feature
-            if (service.getProxyLogLevel() == 2) {
-                //Set a map between inbound and outbound,
-                //so can find inbound by outbound or find outbound by inbound
-                inboundOutboundChannelMap.put(outboundChannel.id(), inboundChannel.id());
-            }
+        });
+    }
 
-            if (!config.isHaProxyProtocolEnabled()) {
-                return;
-            }
+    private String parseHost(String brokerPortAndHost) {

Review comment:
       done




-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on pull request #14713: [Proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode

Posted by GitBox <gi...@apache.org>.
lhotari commented on pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#issuecomment-1071056333


   I asked for feedback in the community meeting from @merlimat . Based on the discussion there was the conclusion that backpressure is needed to prevent a situation where the proxy buffers grow in the case where either the broker connection is slow or the client connection is slow.


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari edited a comment on pull request #14713: [Proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode

Posted by GitBox <gi...@apache.org>.
lhotari edited a comment on pull request #14713:
URL: https://github.com/apache/pulsar/pull/14713#issuecomment-1071114661


   The channel writability change feature in Netty can be configured with `setWriteBufferWaterMark` or `setWriteBufferLowWaterMark`/`setWriteBufferHighWaterMark` methods. By default, the low water mark is 32kB and high water mark is 64kB. That sounds like a sensible default.
   


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org