/
NettyEmitter.java
203 lines (173 loc) · 6.32 KB
/
NettyEmitter.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
package org.apache.s4.comm.netty;
import java.net.ConnectException;
import java.net.InetSocketAddress;
import java.util.concurrent.Executors;
import org.apache.s4.base.Emitter;
import org.apache.s4.comm.topology.ClusterNode;
import org.apache.s4.comm.topology.Topology;
import org.apache.s4.comm.topology.TopologyChangeListener;
import org.jboss.netty.bootstrap.ClientBootstrap;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.buffer.ChannelBuffers;
import org.jboss.netty.channel.Channel;
import org.jboss.netty.channel.ChannelFactory;
import org.jboss.netty.channel.ChannelFuture;
import org.jboss.netty.channel.ChannelFutureListener;
import org.jboss.netty.channel.ChannelHandlerContext;
import org.jboss.netty.channel.ChannelPipeline;
import org.jboss.netty.channel.ChannelPipelineFactory;
import org.jboss.netty.channel.ChannelStateEvent;
import org.jboss.netty.channel.Channels;
import org.jboss.netty.channel.ExceptionEvent;
import org.jboss.netty.channel.SimpleChannelHandler;
import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
import org.jboss.netty.handler.codec.frame.LengthFieldPrepender;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.collect.HashBiMap;
import com.google.inject.Inject;
public class NettyEmitter implements Emitter, ChannelFutureListener,
TopologyChangeListener {
private static final Logger logger = LoggerFactory
.getLogger(NettyEmitter.class);
private Topology topology;
private final ClientBootstrap bootstrap;
// Hashtable inherently allows capturing changes to the underlying topology
private HashBiMap<Integer, Channel> channels;
private HashBiMap<Integer, ClusterNode> nodes;
@Inject
public NettyEmitter(Topology topology) throws InterruptedException {
this.topology = topology;
int clusterSize = this.topology.getTopology().getNodes().size();
channels = HashBiMap.create(clusterSize);
nodes = HashBiMap.create(clusterSize);
for (ClusterNode clusterNode : NettyEmitter.this.topology.getTopology()
.getNodes()) {
Integer partition = clusterNode.getPartition();
nodes.forcePut(partition, clusterNode);
}
ChannelFactory factory = new NioClientSocketChannelFactory(
Executors.newCachedThreadPool(),
Executors.newCachedThreadPool());
bootstrap = new ClientBootstrap(factory);
bootstrap.setPipelineFactory(new ChannelPipelineFactory() {
public ChannelPipeline getPipeline() {
ChannelPipeline p = Channels.pipeline();
p.addLast("1", new LengthFieldPrepender(4));
p.addLast("2", new TestHandler());
return p;
}
});
bootstrap.setOption("tcpNoDelay", true);
bootstrap.setOption("keepAlive", true);
}
private void connectTo(Integer partitionId) {
ClusterNode clusterNode = nodes.get(partitionId);
if (clusterNode == null)
logger.error("No ClusterNode exists for partitionId " + partitionId);
logger.info(String.format("Connecting to %s:%d",
clusterNode.getMachineName(), clusterNode.getPort()));
while (true) {
ChannelFuture f = this.bootstrap.connect(new InetSocketAddress(
clusterNode.getMachineName(), clusterNode.getPort()));
f.awaitUninterruptibly();
if (f.isSuccess()) {
channels.forcePut(partitionId, f.getChannel());
break;
}
try {
Thread.sleep(10);
} catch (InterruptedException ie) {
logger.error(String.format(
"Interrupted while connecting to %s:%d",
clusterNode.getMachineName(), clusterNode.getPort()));
}
}
}
private Object sendLock = new Object();
public void send(int partitionId, byte[] message) {
Channel channel = channels.get(partitionId);
while (channel == null) {
connectTo(partitionId);
channel = channels.get(partitionId); // making sure it is reflected in the map
}
ChannelBuffer buffer = ChannelBuffers.buffer(message.length);
// check if Netty's send queue has gotten quite large
if (!channel.isWritable()) {
synchronized (sendLock) {
// check again now that we have the lock
while (!channel.isWritable()) {
try {
sendLock.wait(); // wait until the channel's queue
// has gone down
} catch (InterruptedException ie) {
return; // somebody wants us to stop running
}
}
// logger.info("Woke up from send block!");
}
}
// between the above isWritable check and the below writeBytes, the
// isWritable
// may become false again. That's OK, we're just trying to avoid a
// very large
// above check to avoid creating a very large send queue inside
// Netty.
buffer.writeBytes(message);
ChannelFuture f = channel.write(buffer);
f.addListener(this);
}
public void operationComplete(ChannelFuture f) {
// when we get here, the I/O operation associated with f is complete
if (f.isCancelled()) {
logger.error("Send I/O was cancelled!! "
+ f.getChannel().getRemoteAddress());
} else if (!f.isSuccess()) {
logger.error("Exception on I/O operation", f.getCause());
// find the partition associated with this broken channel
int partition = channels.inverse().get(f.getChannel());
logger.error(String
.format("I/O on partition %d failed!", partition));
}
}
public void onChange() {
// do nothing for now, don't expect the topology to change.
}
public int getPartitionCount() {
//Number of nodes is not same as number of partitions
return topology.getTopology().getPartitionCount();
}
class TestHandler extends SimpleChannelHandler {
public void channelInterestChanged(ChannelHandlerContext ctx,
ChannelStateEvent e) {
// logger.info(String.format("%08x %08x %08x", e.getValue(),
// e.getChannel().getInterestOps(), Channel.OP_WRITE));
synchronized (sendLock) {
if (e.getChannel().isWritable()) {
sendLock.notify();
}
}
ctx.sendUpstream(e);
}
public void exceptionCaught(ChannelHandlerContext context,
ExceptionEvent event) {
Integer partition = channels.inverse().get(context.getChannel());
if (partition == null) {
logger.error("Error on mystery channel!!");
// return;
}
logger.error("Error on channel to partition " + partition);
try {
throw event.getCause();
} catch (ConnectException ce) {
logger.error(ce.getMessage(), ce);
} catch (Throwable err) {
logger.error("Error", err);
if (context.getChannel().isOpen()) {
logger.error("Closing channel due to exception");
context.getChannel().close();
}
}
}
}
}