Github user revans2 commented on a diff in the pull request:
https://github.com/apache/incubator-storm/pull/103#discussion_r12812760
--- Diff: storm-core/src/jvm/backtype/storm/messaging/netty/Client.java ---
@@ -84,43 +93,87 @@
// Start the connection attempt.
remote_addr = new InetSocketAddress(host, port);
- bootstrap.connect(remote_addr);
+
+ Thread flushChecker = new Thread(new Runnable() {
+ @Override
+ public void run() {
+ //make sure we have a connection
+ connect();
+
+ while(!closing) {
+ long flushCheckTime = flushCheckTimer.get();
+ long now = System.currentTimeMillis();
+ if (now > flushCheckTime) {
+ Channel channel = channelRef.get();
+ if (null != channel && channel.isWritable()) {
+ flush();
+ }
+ }
+ try {
+ Thread.sleep(flushCheckInterval);
+ } catch (InterruptedException e) {
+ break;
+ }
+ }
+
+ }
+ }, name() + "-flush-checker");
+
+ flushChecker.setDaemon(true);
+ flushChecker.start();
}
/**
* We will retry connection with exponential back-off policy
*/
- void reconnect() {
- close_n_release();
-
- //reconnect only if it's not being closed
- if (being_closed.get()) return;
-
- final int tried_count = retries.incrementAndGet();
- if (tried_count <= max_retries) {
- long sleep = getSleepTimeMs();
- LOG.info("Waiting {} ms before trying connection to {}",
sleep, remote_addr);
- TIMER.schedule(new TimerTask() {
- @Override
- public void run() {
- LOG.info("Reconnect ... [{}] to {}", tried_count,
remote_addr);
- bootstrap.connect(remote_addr);
- }}, sleep);
- } else {
- LOG.warn(remote_addr+" is not reachable. We will close this
client.");
- close();
+ private synchronized void connect() {
--- End diff --
I don't like the idea of having connect block until the connection is
established. Any thread that tries to send data to a connection that is still
being established will block until the connection is established. I think it is
more robust to buffer the messages in a data structure and try to handle them
later when the connection is finished.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---