forked from apache/pulsar
/
ConnectionHandler.java
179 lines (156 loc) · 6.92 KB
/
ConnectionHandler.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
/**
* 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.pulsar.client.impl;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLongFieldUpdater;
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.impl.HandlerState.State;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class ConnectionHandler {
private static final AtomicReferenceFieldUpdater<ConnectionHandler, ClientCnx> CLIENT_CNX_UPDATER =
AtomicReferenceFieldUpdater.newUpdater(ConnectionHandler.class, ClientCnx.class, "clientCnx");
@SuppressWarnings("unused")
private volatile ClientCnx clientCnx = null;
protected final HandlerState state;
protected final Backoff backoff;
private static final AtomicLongFieldUpdater<ConnectionHandler> EPOCH_UPDATER = AtomicLongFieldUpdater
.newUpdater(ConnectionHandler.class, "epoch");
private volatile long epoch = 0L;
protected volatile long lastConnectionClosedTimestamp = 0L;
interface Connection {
void connectionFailed(PulsarClientException exception);
void connectionOpened(ClientCnx cnx);
}
protected Connection connection;
protected ConnectionHandler(HandlerState state, Backoff backoff, Connection connection) {
this.state = state;
this.connection = connection;
this.backoff = backoff;
CLIENT_CNX_UPDATER.set(this, null);
}
protected void grabCnx() {
if (CLIENT_CNX_UPDATER.get(this) != null) {
log.warn("[{}] [{}] Client cnx already set, ignoring reconnection request", state.topic, state.getHandlerName());
return;
}
if (!isValidStateForReconnection()) {
// Ignore connection closed when we are shutting down
log.info("[{}] [{}] Ignoring reconnection request (state: {})", state.topic, state.getHandlerName(), state.getState());
return;
}
try {
state.client.getConnection(state.topic) //
.thenAccept(cnx -> connection.connectionOpened(cnx)) //
.exceptionally(this::handleConnectionError);
} catch (Throwable t) {
log.warn("[{}] [{}] Exception thrown while getting connection: ", state.topic, state.getHandlerName(), t);
reconnectLater(t);
}
}
private Void handleConnectionError(Throwable exception) {
log.warn("[{}] [{}] Error connecting to broker: {}", state.topic, state.getHandlerName(), exception.getMessage());
if (exception instanceof PulsarClientException) {
connection.connectionFailed((PulsarClientException) exception);
} else if (exception.getCause() instanceof PulsarClientException) {
connection.connectionFailed((PulsarClientException)exception.getCause());
} else {
connection.connectionFailed(new PulsarClientException(exception));
}
State state = this.state.getState();
if (state == State.Uninitialized || state == State.Connecting || state == State.Ready) {
reconnectLater(exception);
}
return null;
}
protected void reconnectLater(Throwable exception) {
CLIENT_CNX_UPDATER.set(this, null);
if (!isValidStateForReconnection()) {
log.info("[{}] [{}] Ignoring reconnection request (state: {})", state.topic, state.getHandlerName(), state.getState());
return;
}
long delayMs = backoff.next();
log.warn("[{}] [{}] Could not get connection to broker: {} -- Will try again in {} s", state.topic, state.getHandlerName(),
exception.getMessage(), delayMs / 1000.0);
if (state.changeToConnecting()) {
state.client.timer().newTimeout(timeout -> {
log.info("[{}] [{}] Reconnecting after connection was closed", state.topic, state.getHandlerName());
incrementEpoch();
grabCnx();
}, delayMs, TimeUnit.MILLISECONDS);
} else {
log.info("[{}] [{}] Ignoring reconnection request (state: {})",
state.topic, state.getHandlerName(), state.getState());
}
}
protected long incrementEpoch() {
return EPOCH_UPDATER.incrementAndGet(this);
}
public void connectionClosed(ClientCnx cnx) {
lastConnectionClosedTimestamp = System.currentTimeMillis();
state.client.getCnxPool().releaseConnection(cnx);
if (CLIENT_CNX_UPDATER.compareAndSet(this, cnx, null)) {
if (!isValidStateForReconnection()) {
log.info("[{}] [{}] Ignoring reconnection request (state: {})", state.topic, state.getHandlerName(), state.getState());
return;
}
long delayMs = backoff.next();
state.setState(State.Connecting);
log.info("[{}] [{}] Closed connection {} -- Will try again in {} s", state.topic, state.getHandlerName(), cnx.channel(),
delayMs / 1000.0);
state.client.timer().newTimeout(timeout -> {
log.info("[{}] [{}] Reconnecting after timeout", state.topic, state.getHandlerName());
incrementEpoch();
grabCnx();
}, delayMs, TimeUnit.MILLISECONDS);
}
}
protected void resetBackoff() {
backoff.reset();
}
public ClientCnx cnx() {
return CLIENT_CNX_UPDATER.get(this);
}
protected void setClientCnx(ClientCnx clientCnx) {
CLIENT_CNX_UPDATER.set(this, clientCnx);
}
private boolean isValidStateForReconnection() {
State state = this.state.getState();
switch (state) {
case Uninitialized:
case Connecting:
case RegisteringSchema:
case Ready:
// Ok
return true;
case Closing:
case Closed:
case Failed:
case ProducerFenced:
case Terminated:
return false;
}
return false;
}
public long getEpoch() {
return epoch;
}
private static final Logger log = LoggerFactory.getLogger(ConnectionHandler.class);
}