Skip to content

Commit 9db1e1d

Browse files
committed
format code
1 parent 5b39f4e commit 9db1e1d

File tree

2 files changed

+17
-7
lines changed

2 files changed

+17
-7
lines changed

bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -71,6 +71,7 @@
7171
import org.apache.bookkeeper.net.BookieSocketAddress;
7272
import org.apache.bookkeeper.net.DNS;
7373
import org.apache.bookkeeper.processor.RequestProcessor;
74+
import org.apache.bookkeeper.proto.BookieProtocol;
7475
import org.apache.bookkeeper.proto.BookieProtocol.ParsedAddRequest;
7576
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
7677
import org.apache.bookkeeper.proto.RequestStats;

bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestHandler.java

Lines changed: 16 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -27,13 +27,13 @@
2727
import io.netty.channel.ChannelInboundHandlerAdapter;
2828
import io.netty.channel.group.ChannelGroup;
2929
import java.nio.channels.ClosedChannelException;
30-
import lombok.extern.slf4j.Slf4j;
31-
import org.apache.bookkeeper.conf.ServerConfiguration;
32-
import org.apache.bookkeeper.processor.RequestProcessor;
3330
import java.util.ArrayList;
3431
import java.util.List;
3532
import java.util.concurrent.ArrayBlockingQueue;
3633
import java.util.concurrent.BlockingQueue;
34+
import lombok.extern.slf4j.Slf4j;
35+
import org.apache.bookkeeper.conf.ServerConfiguration;
36+
import org.apache.bookkeeper.processor.RequestProcessor;
3737

3838
/**
3939
* Serverside handler for bookkeeper requests.
@@ -42,20 +42,21 @@
4242
public class BookieRequestHandler extends ChannelInboundHandlerAdapter {
4343

4444
static final Object EVENT_FLUSH_ALL_PENDING_RESPONSES = new Object();
45+
private static final int CAPACITY = 10_000;
4546

4647
private final RequestProcessor requestProcessor;
4748
private final ChannelGroup allChannels;
4849

4950
private ChannelHandlerContext ctx;
50-
private BlockingQueue<BookieProtocol.ParsedAddRequest> msgs;
51+
private final BlockingQueue<BookieProtocol.ParsedAddRequest> msgs;
5152

5253
private ByteBuf pendingSendResponses = null;
5354
private int maxPendingResponsesSize;
5455

5556
BookieRequestHandler(ServerConfiguration conf, RequestProcessor processor, ChannelGroup allChannels) {
5657
this.requestProcessor = processor;
5758
this.allChannels = allChannels;
58-
this.msgs = new ArrayBlockingQueue<>(10_000);
59+
this.msgs = new ArrayBlockingQueue<>(CAPACITY);
5960
}
6061

6162
public ChannelHandlerContext ctx() {
@@ -102,6 +103,13 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception
102103
&& ((BookieProtocol.ParsedAddRequest) msg).getProtocolVersion() == BookieProtocol.CURRENT_PROTOCOL_VERSION
103104
&& !((BookieProtocol.ParsedAddRequest) msg).isRecoveryAdd()) {
104105
msgs.put((BookieProtocol.ParsedAddRequest) msg);
106+
107+
if (msgs.size() == CAPACITY) {
108+
int count = msgs.size();
109+
List<BookieProtocol.ParsedAddRequest> c = new ArrayList<>(count);
110+
msgs.drainTo(c, count);
111+
requestProcessor.processAddRequest(c, this);
112+
}
105113
} else {
106114
requestProcessor.processRequest(msg, this);
107115
}
@@ -110,8 +118,9 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception
110118
@Override
111119
public void channelReadComplete(ChannelHandlerContext ctx) {
112120
if (!msgs.isEmpty()) {
113-
List<BookieProtocol.ParsedAddRequest> c = new ArrayList<>();
114-
msgs.drainTo(c);
121+
int count = msgs.size();
122+
List<BookieProtocol.ParsedAddRequest> c = new ArrayList<>(count);
123+
msgs.drainTo(c, count);
115124
requestProcessor.processAddRequest(c, this);
116125
}
117126
}

0 commit comments

Comments
 (0)