-
Notifications
You must be signed in to change notification settings - Fork 1.6k
/
ComputeNode.java
419 lines (339 loc) · 12.2 KB
/
ComputeNode.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
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
// This file is licensed under the Elastic License 2.0. Copyright 2021-present, StarRocks Limited.
package com.starrocks.system;
import com.google.gson.annotations.SerializedName;
import com.starrocks.alter.DecommissionBackendJob;
import com.starrocks.common.Config;
import com.starrocks.common.io.Text;
import com.starrocks.common.io.Writable;
import com.starrocks.persist.gson.GsonUtils;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* This class extends the primary identifier of a compute node with computing capabilities
* and no storage capacity。
*/
public class ComputeNode implements IComputable, Writable {
private static final Logger LOG = LogManager.getLogger(ComputeNode.class);
@SerializedName("id")
private long id;
@SerializedName("host")
private String host;
@SerializedName("version")
private String version;
@SerializedName("heartbeatPort")
private int heartbeatPort; // heartbeat
@SerializedName("bePort")
private volatile int bePort; // be
@SerializedName("httpPort")
private volatile int httpPort; // web service
@SerializedName("beRpcPort")
private volatile int beRpcPort; // be rpc port
@SerializedName("brpcPort")
private volatile int brpcPort = -1;
@SerializedName("cpuCores")
private volatile int cpuCores = 0; // Cpu cores of node
@SerializedName("lastUpdateMs")
private volatile long lastUpdateMs;
@SerializedName("lastStartTime")
private volatile long lastStartTime;
@SerializedName("isAlive")
private AtomicBoolean isAlive;
@SerializedName("isDecommissioned")
private final AtomicBoolean isDecommissioned;
@SerializedName("decommissionType")
private volatile int decommissionType;
@SerializedName("ownerClusterName")
private volatile String ownerClusterName;
// to index the state in some cluster
@SerializedName("backendState")
private volatile int backendState;
// private BackendState backendState;
@SerializedName("heartbeatErrMsg")
private String heartbeatErrMsg = "";
@SerializedName("lastMissingHeartbeatTime")
private long lastMissingHeartbeatTime = -1;
@SerializedName("heartbeatRetryTimes")
private int heartbeatRetryTimes = 0;
// port of starlet on BE
private volatile int starletPort;
public ComputeNode() {
this.host = "";
this.version = "";
this.lastUpdateMs = 0;
this.lastStartTime = 0;
this.isAlive = new AtomicBoolean();
this.isDecommissioned = new AtomicBoolean(false);
this.bePort = 0;
this.httpPort = 0;
this.beRpcPort = 0;
this.ownerClusterName = "";
this.backendState = Backend.BackendState.free.ordinal();
this.decommissionType = DecommissionBackendJob.DecommissionType.SystemDecommission.ordinal();
}
public ComputeNode(long id, String host, int heartbeatPort) {
this.id = id;
this.host = host;
this.version = "";
this.heartbeatPort = heartbeatPort;
this.bePort = -1;
this.httpPort = -1;
this.beRpcPort = -1;
this.lastUpdateMs = -1L;
this.lastStartTime = -1L;
this.isAlive = new AtomicBoolean(false);
this.isDecommissioned = new AtomicBoolean(false);
this.ownerClusterName = "";
this.backendState = Backend.BackendState.free.ordinal();
this.decommissionType = DecommissionBackendJob.DecommissionType.SystemDecommission.ordinal();
}
public int getStarletPort() {
return starletPort;
}
// for test only
public void setStarletPort(int starletPort) {
this.starletPort = starletPort;
}
public long getId() {
return id;
}
public String getHost() {
return host;
}
public String getVersion() {
return version;
}
public int getBePort() {
return bePort;
}
public int getHeartbeatPort() {
return heartbeatPort;
}
public int getHttpPort() {
return httpPort;
}
public int getBeRpcPort() {
return beRpcPort;
}
public int getBrpcPort() {
return brpcPort;
}
public String getHeartbeatErrMsg() {
return heartbeatErrMsg;
}
// for test only
public void updateOnce(int bePort, int httpPort, int beRpcPort) {
if (this.bePort != bePort) {
this.bePort = bePort;
}
if (this.httpPort != httpPort) {
this.httpPort = httpPort;
}
if (this.beRpcPort != beRpcPort) {
this.beRpcPort = beRpcPort;
}
long currentTime = System.currentTimeMillis();
this.lastUpdateMs = currentTime;
if (!isAlive.get()) {
this.lastStartTime = currentTime;
LOG.info("{} is alive,", this.toString());
this.isAlive.set(true);
}
heartbeatErrMsg = "";
}
public boolean setDecommissioned(boolean isDecommissioned) {
if (this.isDecommissioned.compareAndSet(!isDecommissioned, isDecommissioned)) {
LOG.warn("{} set decommission: {}", this.toString(), isDecommissioned);
return true;
}
return false;
}
public void setId(long id) {
this.id = id;
}
public void setHost(String host) {
this.host = host;
}
public void setBackendState(Backend.BackendState state) {
this.backendState = state.ordinal();
}
protected void setHeartbeatPort(int heartbeatPort) {
this.heartbeatPort = heartbeatPort;
}
public void setAlive(boolean isAlive) {
this.isAlive.set(isAlive);
}
public void setBePort(int agentPort) {
this.bePort = agentPort;
}
public void setHttpPort(int httpPort) {
this.httpPort = httpPort;
}
public void setBeRpcPort(int beRpcPort) {
this.beRpcPort = beRpcPort;
}
public void setBrpcPort(int brpcPort) {
this.brpcPort = brpcPort;
}
public long getLastUpdateMs() {
return this.lastUpdateMs;
}
public void setLastUpdateMs(long currentTime) {
this.lastUpdateMs = currentTime;
}
public long getLastStartTime() {
return this.lastStartTime;
}
public void setLastStartTime(long currentTime) {
this.lastStartTime = currentTime;
}
public long getLastMissingHeartbeatTime() {
return lastMissingHeartbeatTime;
}
public boolean isAlive() {
return this.isAlive.get();
}
public boolean isDecommissioned() {
return this.isDecommissioned.get();
}
public boolean isAvailable() {
return this.isAlive.get() && !this.isDecommissioned.get();
}
@Override
public void write(DataOutput out) throws IOException {
String s = GsonUtils.GSON.toJson(this);
Text.writeString(out, s);
}
public static ComputeNode read(DataInput in) throws IOException {
String json = Text.readString(in);
return GsonUtils.GSON.fromJson(json, ComputeNode.class);
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (!(obj instanceof ComputeNode)) {
return false;
}
ComputeNode computeNode = (ComputeNode) obj;
return (id == computeNode.id) && (host.equals(computeNode.host)) && (heartbeatPort == computeNode.heartbeatPort)
&& (bePort == computeNode.bePort) && (isAlive.get() == computeNode.isAlive.get());
}
@Override
public String toString() {
return "ComputeNode [id=" + id + ", host=" + host + ", heartbeatPort=" + heartbeatPort + ", alive=" +
isAlive.get() + "]";
}
public String getOwnerClusterName() {
return ownerClusterName;
}
public void setOwnerClusterName(String name) {
ownerClusterName = name;
}
public void clearClusterName() {
ownerClusterName = "";
}
public Backend.BackendState getBackendState() {
switch (backendState) {
case 0:
return Backend.BackendState.using;
case 1:
return Backend.BackendState.offline;
default:
return Backend.BackendState.free;
}
}
public void setDecommissionType(DecommissionBackendJob.DecommissionType type) {
decommissionType = type.ordinal();
}
public void setVersion(String version) {
this.version = version;
}
public AtomicBoolean getIsAlive() {
return isAlive;
}
public void setIsAlive(AtomicBoolean isAlive) {
this.isAlive = isAlive;
}
public AtomicBoolean getIsDecommissioned() {
return isDecommissioned;
}
public void setDecommissionType(int decommissionType) {
this.decommissionType = decommissionType;
}
public void setBackendState(int backendState) {
this.backendState = backendState;
}
public DecommissionBackendJob.DecommissionType getDecommissionType() {
if (decommissionType == DecommissionBackendJob.DecommissionType.ClusterDecommission.ordinal()) {
return DecommissionBackendJob.DecommissionType.ClusterDecommission;
}
return DecommissionBackendJob.DecommissionType.SystemDecommission;
}
/**
* handle Compute node's heartbeat response.
* return true if any port changed, or alive state is changed.
*/
public boolean handleHbResponse(BackendHbResponse hbResponse) {
boolean isChanged = false;
if (hbResponse.getStatus() == HeartbeatResponse.HbStatus.OK) {
if (!this.version.equals(hbResponse.getVersion())) {
isChanged = true;
this.version = hbResponse.getVersion();
}
if (this.bePort != hbResponse.getBePort()) {
isChanged = true;
this.bePort = hbResponse.getBePort();
}
if (this.httpPort != hbResponse.getHttpPort()) {
isChanged = true;
this.httpPort = hbResponse.getHttpPort();
}
if (this.brpcPort != hbResponse.getBrpcPort()) {
isChanged = true;
this.brpcPort = hbResponse.getBrpcPort();
}
if (Config.integrate_starmgr && this.starletPort != hbResponse.getStarletPort()) {
isChanged = true;
this.starletPort = hbResponse.getStarletPort();
}
this.lastUpdateMs = hbResponse.getHbTime();
if (!isAlive.get()) {
isChanged = true;
this.lastStartTime = hbResponse.getHbTime();
LOG.info("{} is alive, last start time: {}", this.toString(), hbResponse.getHbTime());
this.isAlive.set(true);
} else if (this.lastStartTime <= 0) {
this.lastStartTime = hbResponse.getHbTime();
}
if (this.cpuCores != hbResponse.getCpuCores()) {
isChanged = true;
this.cpuCores = hbResponse.getCpuCores();
BackendCoreStat.setNumOfHardwareCoresOfBe(hbResponse.getBeId(), hbResponse.getCpuCores());
}
heartbeatErrMsg = "";
this.heartbeatRetryTimes = 0;
} else {
if (this.heartbeatRetryTimes < Config.heartbeat_retry_times) {
this.heartbeatRetryTimes++;
} else {
if (isAlive.compareAndSet(true, false)) {
LOG.info("{} is dead,", this.toString());
}
heartbeatErrMsg = hbResponse.getMsg() == null ? "Unknown error" : hbResponse.getMsg();
lastMissingHeartbeatTime = System.currentTimeMillis();
}
// When the master receives an error heartbeat info which status not ok,
// this heartbeat info also need to be synced to follower.
// Since the failed heartbeat info also modifies fe's memory, (this.heartbeatRetryTimes++;)
// if this heartbeat is not synchronized to the follower,
// that will cause the Follower and master’s memory to be inconsistent
isChanged = true;
}
return isChanged;
}
}