Replace fastjson with jackson for nacos-naming controller and consistency
This commit is contained in:
parent
eec067ffbd
commit
3b0e9df9a9
@ -15,7 +15,7 @@
|
|||||||
*/
|
*/
|
||||||
package com.alibaba.nacos.naming.consistency.ephemeral.distro;
|
package com.alibaba.nacos.naming.consistency.ephemeral.distro;
|
||||||
|
|
||||||
import com.alibaba.fastjson.JSON;
|
import com.alibaba.nacos.common.utils.JacksonUtils;
|
||||||
import com.alibaba.nacos.core.cluster.Member;
|
import com.alibaba.nacos.core.cluster.Member;
|
||||||
import com.alibaba.nacos.naming.misc.*;
|
import com.alibaba.nacos.naming.misc.*;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
@ -125,7 +125,7 @@ public class TaskDispatcher {
|
|||||||
syncTask.setTargetServer(member.getAddress());
|
syncTask.setTargetServer(member.getAddress());
|
||||||
|
|
||||||
if (Loggers.DISTRO.isDebugEnabled() && StringUtils.isNotBlank(key)) {
|
if (Loggers.DISTRO.isDebugEnabled() && StringUtils.isNotBlank(key)) {
|
||||||
Loggers.DISTRO.debug("add sync task: {}", JSON.toJSONString(syncTask));
|
Loggers.DISTRO.debug("add sync task: {}", JacksonUtils.toJson(syncTask));
|
||||||
}
|
}
|
||||||
|
|
||||||
dataSyncer.submit(syncTask, 0);
|
dataSyncer.submit(syncTask, 0);
|
||||||
|
@ -15,10 +15,7 @@
|
|||||||
*/
|
*/
|
||||||
package com.alibaba.nacos.naming.consistency.persistent.raft;
|
package com.alibaba.nacos.naming.consistency.persistent.raft;
|
||||||
|
|
||||||
import com.alibaba.fastjson.JSON;
|
import com.alibaba.nacos.common.utils.JacksonUtils;
|
||||||
import com.alibaba.fastjson.JSONArray;
|
|
||||||
import com.alibaba.fastjson.JSONObject;
|
|
||||||
import com.alibaba.fastjson.TypeReference;
|
|
||||||
import com.alibaba.nacos.core.utils.ApplicationUtils;
|
import com.alibaba.nacos.core.utils.ApplicationUtils;
|
||||||
import com.alibaba.nacos.naming.consistency.ApplyAction;
|
import com.alibaba.nacos.naming.consistency.ApplyAction;
|
||||||
import com.alibaba.nacos.naming.consistency.Datum;
|
import com.alibaba.nacos.naming.consistency.Datum;
|
||||||
@ -29,6 +26,10 @@ import com.alibaba.nacos.naming.core.Service;
|
|||||||
import com.alibaba.nacos.naming.misc.*;
|
import com.alibaba.nacos.naming.misc.*;
|
||||||
import com.alibaba.nacos.naming.monitor.MetricsMonitor;
|
import com.alibaba.nacos.naming.monitor.MetricsMonitor;
|
||||||
import com.alibaba.nacos.naming.pojo.Record;
|
import com.alibaba.nacos.naming.pojo.Record;
|
||||||
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
|
import com.fasterxml.jackson.databind.JsonNode;
|
||||||
|
import com.fasterxml.jackson.databind.node.ArrayNode;
|
||||||
|
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||||
import com.ning.http.client.AsyncCompletionHandler;
|
import com.ning.http.client.AsyncCompletionHandler;
|
||||||
import com.ning.http.client.Response;
|
import com.ning.http.client.Response;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
@ -155,15 +156,15 @@ public class RaftCore {
|
|||||||
public void signalPublish(String key, Record value) throws Exception {
|
public void signalPublish(String key, Record value) throws Exception {
|
||||||
|
|
||||||
if (!isLeader()) {
|
if (!isLeader()) {
|
||||||
JSONObject params = new JSONObject();
|
ObjectNode params = JacksonUtils.createEmptyJsonNode();
|
||||||
params.put("key", key);
|
params.put("key", key);
|
||||||
params.put("value", value);
|
params.replace("value", JacksonUtils.transferToJsonNode(value));
|
||||||
Map<String, String> parameters = new HashMap<>(1);
|
Map<String, String> parameters = new HashMap<>(1);
|
||||||
parameters.put("key", key);
|
parameters.put("key", key);
|
||||||
|
|
||||||
final RaftPeer leader = getLeader();
|
final RaftPeer leader = getLeader();
|
||||||
|
|
||||||
raftProxy.proxyPostLarge(leader.ip, API_PUB, params.toJSONString(), parameters);
|
raftProxy.proxyPostLarge(leader.ip, API_PUB, params.toString(), parameters);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -179,13 +180,13 @@ public class RaftCore {
|
|||||||
datum.timestamp.set(getDatum(key).timestamp.incrementAndGet());
|
datum.timestamp.set(getDatum(key).timestamp.incrementAndGet());
|
||||||
}
|
}
|
||||||
|
|
||||||
JSONObject json = new JSONObject();
|
ObjectNode json = JacksonUtils.createEmptyJsonNode();
|
||||||
json.put("datum", datum);
|
json.replace("datum", JacksonUtils.transferToJsonNode(datum));
|
||||||
json.put("source", peers.local());
|
json.replace("source", JacksonUtils.transferToJsonNode(peers.local()));
|
||||||
|
|
||||||
onPublish(datum, peers.local());
|
onPublish(datum, peers.local());
|
||||||
|
|
||||||
final String content = JSON.toJSONString(json);
|
final String content = json.toString();
|
||||||
|
|
||||||
final CountDownLatch latch = new CountDownLatch(peers.majorityCount());
|
final CountDownLatch latch = new CountDownLatch(peers.majorityCount());
|
||||||
for (final String server : peers.allServersIncludeMyself()) {
|
for (final String server : peers.allServersIncludeMyself()) {
|
||||||
@ -239,18 +240,18 @@ public class RaftCore {
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
JSONObject json = new JSONObject();
|
|
||||||
// construct datum:
|
// construct datum:
|
||||||
Datum datum = new Datum();
|
Datum datum = new Datum();
|
||||||
datum.key = key;
|
datum.key = key;
|
||||||
json.put("datum", datum);
|
ObjectNode json = JacksonUtils.createEmptyJsonNode();
|
||||||
json.put("source", peers.local());
|
json.replace("datum", JacksonUtils.transferToJsonNode(datum));
|
||||||
|
json.replace("source", JacksonUtils.transferToJsonNode(peers.local()));
|
||||||
|
|
||||||
onDelete(datum.key, peers.local());
|
onDelete(datum.key, peers.local());
|
||||||
|
|
||||||
for (final String server : peers.allServersWithoutMySelf()) {
|
for (final String server : peers.allServersWithoutMySelf()) {
|
||||||
String url = buildURL(server, API_ON_DEL);
|
String url = buildURL(server, API_ON_DEL);
|
||||||
HttpClient.asyncHttpDeleteLarge(url, null, JSON.toJSONString(json)
|
HttpClient.asyncHttpDeleteLarge(url, null, json.toString()
|
||||||
, new AsyncCompletionHandler<Integer>() {
|
, new AsyncCompletionHandler<Integer>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer onCompleted(Response response) throws Exception {
|
public Integer onCompleted(Response response) throws Exception {
|
||||||
@ -281,14 +282,14 @@ public class RaftCore {
|
|||||||
|
|
||||||
if (!peers.isLeader(source.ip)) {
|
if (!peers.isLeader(source.ip)) {
|
||||||
Loggers.RAFT.warn("peer {} tried to publish data but wasn't leader, leader: {}",
|
Loggers.RAFT.warn("peer {} tried to publish data but wasn't leader, leader: {}",
|
||||||
JSON.toJSONString(source), JSON.toJSONString(getLeader()));
|
JacksonUtils.toJson(source), JacksonUtils.toJson(getLeader()));
|
||||||
throw new IllegalStateException("peer(" + source.ip + ") tried to publish " +
|
throw new IllegalStateException("peer(" + source.ip + ") tried to publish " +
|
||||||
"data but wasn't leader");
|
"data but wasn't leader");
|
||||||
}
|
}
|
||||||
|
|
||||||
if (source.term.get() < local.term.get()) {
|
if (source.term.get() < local.term.get()) {
|
||||||
Loggers.RAFT.warn("out of date publish, pub-term: {}, cur-term: {}",
|
Loggers.RAFT.warn("out of date publish, pub-term: {}, cur-term: {}",
|
||||||
JSON.toJSONString(source), JSON.toJSONString(local));
|
JacksonUtils.toJson(source), JacksonUtils.toJson(local));
|
||||||
throw new IllegalStateException("out of date publish, pub-term:"
|
throw new IllegalStateException("out of date publish, pub-term:"
|
||||||
+ source.term.get() + ", cur-term: " + local.term.get());
|
+ source.term.get() + ", cur-term: " + local.term.get());
|
||||||
}
|
}
|
||||||
@ -326,13 +327,13 @@ public class RaftCore {
|
|||||||
|
|
||||||
if (!peers.isLeader(source.ip)) {
|
if (!peers.isLeader(source.ip)) {
|
||||||
Loggers.RAFT.warn("peer {} tried to publish data but wasn't leader, leader: {}",
|
Loggers.RAFT.warn("peer {} tried to publish data but wasn't leader, leader: {}",
|
||||||
JSON.toJSONString(source), JSON.toJSONString(getLeader()));
|
JacksonUtils.toJson(source), JacksonUtils.toJson(getLeader()));
|
||||||
throw new IllegalStateException("peer(" + source.ip + ") tried to publish data but wasn't leader");
|
throw new IllegalStateException("peer(" + source.ip + ") tried to publish data but wasn't leader");
|
||||||
}
|
}
|
||||||
|
|
||||||
if (source.term.get() < local.term.get()) {
|
if (source.term.get() < local.term.get()) {
|
||||||
Loggers.RAFT.warn("out of date publish, pub-term: {}, cur-term: {}",
|
Loggers.RAFT.warn("out of date publish, pub-term: {}, cur-term: {}",
|
||||||
JSON.toJSONString(source), JSON.toJSONString(local));
|
JacksonUtils.toJson(source), JacksonUtils.toJson(local));
|
||||||
throw new IllegalStateException("out of date publish, pub-term:"
|
throw new IllegalStateException("out of date publish, pub-term:"
|
||||||
+ source.term + ", cur-term: " + local.term);
|
+ source.term + ", cur-term: " + local.term);
|
||||||
}
|
}
|
||||||
@ -391,7 +392,7 @@ public class RaftCore {
|
|||||||
|
|
||||||
RaftPeer local = peers.get(NetUtils.localServer());
|
RaftPeer local = peers.get(NetUtils.localServer());
|
||||||
Loggers.RAFT.info("leader timeout, start voting,leader: {}, term: {}",
|
Loggers.RAFT.info("leader timeout, start voting,leader: {}, term: {}",
|
||||||
JSON.toJSONString(getLeader()), local.term);
|
JacksonUtils.toJson(getLeader()), local.term);
|
||||||
|
|
||||||
peers.reset();
|
peers.reset();
|
||||||
|
|
||||||
@ -400,7 +401,7 @@ public class RaftCore {
|
|||||||
local.state = RaftPeer.State.CANDIDATE;
|
local.state = RaftPeer.State.CANDIDATE;
|
||||||
|
|
||||||
Map<String, String> params = new HashMap<>(1);
|
Map<String, String> params = new HashMap<>(1);
|
||||||
params.put("vote", JSON.toJSONString(local));
|
params.put("vote", JacksonUtils.toJson(local));
|
||||||
for (final String server : peers.allServersWithoutMySelf()) {
|
for (final String server : peers.allServersWithoutMySelf()) {
|
||||||
final String url = buildURL(server, API_VOTE);
|
final String url = buildURL(server, API_VOTE);
|
||||||
try {
|
try {
|
||||||
@ -412,9 +413,9 @@ public class RaftCore {
|
|||||||
return 1;
|
return 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
RaftPeer peer = JSON.parseObject(response.getResponseBody(), RaftPeer.class);
|
RaftPeer peer = JacksonUtils.toObj(response.getResponseBody(), RaftPeer.class);
|
||||||
|
|
||||||
Loggers.RAFT.info("received approve from peer: {}", JSON.toJSONString(peer));
|
Loggers.RAFT.info("received approve from peer: {}", JacksonUtils.toJson(peer));
|
||||||
|
|
||||||
peers.decideLeader(peer);
|
peers.decideLeader(peer);
|
||||||
|
|
||||||
@ -494,10 +495,10 @@ public class RaftCore {
|
|||||||
local.resetLeaderDue();
|
local.resetLeaderDue();
|
||||||
|
|
||||||
// build data
|
// build data
|
||||||
JSONObject packet = new JSONObject();
|
ObjectNode packet = JacksonUtils.createEmptyJsonNode();
|
||||||
packet.put("peer", local);
|
packet.replace("peer", JacksonUtils.transferToJsonNode(local));
|
||||||
|
|
||||||
JSONArray array = new JSONArray();
|
ArrayNode array = JacksonUtils.createEmptyArrayNode();
|
||||||
|
|
||||||
if (switchDomain.isSendBeatOnly()) {
|
if (switchDomain.isSendBeatOnly()) {
|
||||||
Loggers.RAFT.info("[SEND-BEAT-ONLY] {}", String.valueOf(switchDomain.isSendBeatOnly()));
|
Loggers.RAFT.info("[SEND-BEAT-ONLY] {}", String.valueOf(switchDomain.isSendBeatOnly()));
|
||||||
@ -506,25 +507,25 @@ public class RaftCore {
|
|||||||
if (!switchDomain.isSendBeatOnly()) {
|
if (!switchDomain.isSendBeatOnly()) {
|
||||||
for (Datum datum : datums.values()) {
|
for (Datum datum : datums.values()) {
|
||||||
|
|
||||||
JSONObject element = new JSONObject();
|
ObjectNode element = JacksonUtils.createEmptyJsonNode();
|
||||||
|
|
||||||
if (KeyBuilder.matchServiceMetaKey(datum.key)) {
|
if (KeyBuilder.matchServiceMetaKey(datum.key)) {
|
||||||
element.put("key", KeyBuilder.briefServiceMetaKey(datum.key));
|
element.put("key", KeyBuilder.briefServiceMetaKey(datum.key));
|
||||||
} else if (KeyBuilder.matchInstanceListKey(datum.key)) {
|
} else if (KeyBuilder.matchInstanceListKey(datum.key)) {
|
||||||
element.put("key", KeyBuilder.briefInstanceListkey(datum.key));
|
element.put("key", KeyBuilder.briefInstanceListkey(datum.key));
|
||||||
}
|
}
|
||||||
element.put("timestamp", datum.timestamp);
|
element.put("timestamp", datum.timestamp.get());
|
||||||
|
|
||||||
array.add(element);
|
array.add(element);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
packet.put("datums", array);
|
packet.replace("datums", array);
|
||||||
// broadcast
|
// broadcast
|
||||||
Map<String, String> params = new HashMap<String, String>(1);
|
Map<String, String> params = new HashMap<String, String>(1);
|
||||||
params.put("beat", JSON.toJSONString(packet));
|
params.put("beat", JacksonUtils.toJson(packet));
|
||||||
|
|
||||||
String content = JSON.toJSONString(params);
|
String content = JacksonUtils.toJson(params);
|
||||||
|
|
||||||
ByteArrayOutputStream out = new ByteArrayOutputStream();
|
ByteArrayOutputStream out = new ByteArrayOutputStream();
|
||||||
GZIPOutputStream gzip = new GZIPOutputStream(out);
|
GZIPOutputStream gzip = new GZIPOutputStream(out);
|
||||||
@ -555,7 +556,7 @@ public class RaftCore {
|
|||||||
return 1;
|
return 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
peers.update(JSON.parseObject(response.getResponseBody(), RaftPeer.class));
|
peers.update(JacksonUtils.toObj(response.getResponseBody(), RaftPeer.class));
|
||||||
if (Loggers.RAFT.isDebugEnabled()) {
|
if (Loggers.RAFT.isDebugEnabled()) {
|
||||||
Loggers.RAFT.debug("receive beat response from: {}", url);
|
Loggers.RAFT.debug("receive beat response from: {}", url);
|
||||||
}
|
}
|
||||||
@ -577,38 +578,39 @@ public class RaftCore {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public RaftPeer receivedBeat(JSONObject beat) throws Exception {
|
public RaftPeer receivedBeat(JsonNode beat) throws Exception {
|
||||||
final RaftPeer local = peers.local();
|
final RaftPeer local = peers.local();
|
||||||
final RaftPeer remote = new RaftPeer();
|
final RaftPeer remote = new RaftPeer();
|
||||||
remote.ip = beat.getJSONObject("peer").getString("ip");
|
JsonNode peer = beat.get("peer");
|
||||||
remote.state = RaftPeer.State.valueOf(beat.getJSONObject("peer").getString("state"));
|
remote.ip = peer.get("ip").asText();
|
||||||
remote.term.set(beat.getJSONObject("peer").getLongValue("term"));
|
remote.state = RaftPeer.State.valueOf(peer.get("state").asText());
|
||||||
remote.heartbeatDueMs = beat.getJSONObject("peer").getLongValue("heartbeatDueMs");
|
remote.term.set(peer.get("term").asLong());
|
||||||
remote.leaderDueMs = beat.getJSONObject("peer").getLongValue("leaderDueMs");
|
remote.heartbeatDueMs = peer.get("heartbeatDueMs").asLong();
|
||||||
remote.voteFor = beat.getJSONObject("peer").getString("voteFor");
|
remote.leaderDueMs = peer.get("leaderDueMs").asLong();
|
||||||
|
remote.voteFor = peer.get("voteFor").asText();
|
||||||
|
|
||||||
if (remote.state != RaftPeer.State.LEADER) {
|
if (remote.state != RaftPeer.State.LEADER) {
|
||||||
Loggers.RAFT.info("[RAFT] invalid state from master, state: {}, remote peer: {}",
|
Loggers.RAFT.info("[RAFT] invalid state from master, state: {}, remote peer: {}",
|
||||||
remote.state, JSON.toJSONString(remote));
|
remote.state, JacksonUtils.toJson(remote));
|
||||||
throw new IllegalArgumentException("invalid state from master, state: " + remote.state);
|
throw new IllegalArgumentException("invalid state from master, state: " + remote.state);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (local.term.get() > remote.term.get()) {
|
if (local.term.get() > remote.term.get()) {
|
||||||
Loggers.RAFT.info("[RAFT] out of date beat, beat-from-term: {}, beat-to-term: {}, remote peer: {}, and leaderDueMs: {}"
|
Loggers.RAFT.info("[RAFT] out of date beat, beat-from-term: {}, beat-to-term: {}, remote peer: {}, and leaderDueMs: {}"
|
||||||
, remote.term.get(), local.term.get(), JSON.toJSONString(remote), local.leaderDueMs);
|
, remote.term.get(), local.term.get(), JacksonUtils.toJson(remote), local.leaderDueMs);
|
||||||
throw new IllegalArgumentException("out of date beat, beat-from-term: " + remote.term.get()
|
throw new IllegalArgumentException("out of date beat, beat-from-term: " + remote.term.get()
|
||||||
+ ", beat-to-term: " + local.term.get());
|
+ ", beat-to-term: " + local.term.get());
|
||||||
}
|
}
|
||||||
|
|
||||||
if (local.state != RaftPeer.State.FOLLOWER) {
|
if (local.state != RaftPeer.State.FOLLOWER) {
|
||||||
|
|
||||||
Loggers.RAFT.info("[RAFT] make remote as leader, remote peer: {}", JSON.toJSONString(remote));
|
Loggers.RAFT.info("[RAFT] make remote as leader, remote peer: {}", JacksonUtils.toJson(remote));
|
||||||
// mk follower
|
// mk follower
|
||||||
local.state = RaftPeer.State.FOLLOWER;
|
local.state = RaftPeer.State.FOLLOWER;
|
||||||
local.voteFor = remote.ip;
|
local.voteFor = remote.ip;
|
||||||
}
|
}
|
||||||
|
|
||||||
final JSONArray beatDatums = beat.getJSONArray("datums");
|
final JsonNode beatDatums = beat.get("datums");
|
||||||
local.resetLeaderDue();
|
local.resetLeaderDue();
|
||||||
local.resetHeartbeatDue();
|
local.resetHeartbeatDue();
|
||||||
|
|
||||||
@ -633,8 +635,8 @@ public class RaftCore {
|
|||||||
for (Object object : beatDatums) {
|
for (Object object : beatDatums) {
|
||||||
processedCount = processedCount + 1;
|
processedCount = processedCount + 1;
|
||||||
|
|
||||||
JSONObject entry = (JSONObject) object;
|
JsonNode entry = (JsonNode) object;
|
||||||
String key = entry.getString("key");
|
String key = entry.get("key").asText();
|
||||||
final String datumKey;
|
final String datumKey;
|
||||||
|
|
||||||
if (KeyBuilder.matchServiceMetaKey(key)) {
|
if (KeyBuilder.matchServiceMetaKey(key)) {
|
||||||
@ -646,7 +648,7 @@ public class RaftCore {
|
|||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
long timestamp = entry.getLong("timestamp");
|
long timestamp = entry.get("timestamp").asLong();
|
||||||
|
|
||||||
receivedKeysMap.put(datumKey, 1);
|
receivedKeysMap.put(datumKey, 1);
|
||||||
|
|
||||||
@ -681,37 +683,34 @@ public class RaftCore {
|
|||||||
return 1;
|
return 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
List<JSONObject> datumList = JSON.parseObject(response.getResponseBody(), new TypeReference<List<JSONObject>>() {
|
List<JsonNode> datumList = JacksonUtils.toObj(response.getResponseBody(), new TypeReference<List<JsonNode>>() {});
|
||||||
});
|
|
||||||
|
|
||||||
for (JSONObject datumJson : datumList) {
|
for (JsonNode datumJson : datumList) {
|
||||||
OPERATE_LOCK.lock();
|
OPERATE_LOCK.lock();
|
||||||
Datum newDatum = null;
|
Datum newDatum = null;
|
||||||
try {
|
try {
|
||||||
|
|
||||||
Datum oldDatum = getDatum(datumJson.getString("key"));
|
Datum oldDatum = getDatum(datumJson.get("key").asText());
|
||||||
|
|
||||||
if (oldDatum != null && datumJson.getLongValue("timestamp") <= oldDatum.timestamp.get()) {
|
if (oldDatum != null && datumJson.get("timestamp").asLong() <= oldDatum.timestamp.get()) {
|
||||||
Loggers.RAFT.info("[NACOS-RAFT] timestamp is smaller than that of mine, key: {}, remote: {}, local: {}",
|
Loggers.RAFT.info("[NACOS-RAFT] timestamp is smaller than that of mine, key: {}, remote: {}, local: {}",
|
||||||
datumJson.getString("key"), datumJson.getLongValue("timestamp"), oldDatum.timestamp);
|
datumJson.get("key").asText(), datumJson.get("timestamp").asLong(), oldDatum.timestamp);
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (KeyBuilder.matchServiceMetaKey(datumJson.getString("key"))) {
|
if (KeyBuilder.matchServiceMetaKey(datumJson.get("key").asText())) {
|
||||||
Datum<Service> serviceDatum = new Datum<>();
|
Datum<Service> serviceDatum = new Datum<>();
|
||||||
serviceDatum.key = datumJson.getString("key");
|
serviceDatum.key = datumJson.get("key").asText();
|
||||||
serviceDatum.timestamp.set(datumJson.getLongValue("timestamp"));
|
serviceDatum.timestamp.set(datumJson.get("timestamp").asLong());
|
||||||
serviceDatum.value =
|
serviceDatum.value = JacksonUtils.toObj(datumJson.get("value").toString(), Service.class);
|
||||||
JSON.parseObject(JSON.toJSONString(datumJson.getJSONObject("value")), Service.class);
|
|
||||||
newDatum = serviceDatum;
|
newDatum = serviceDatum;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (KeyBuilder.matchInstanceListKey(datumJson.getString("key"))) {
|
if (KeyBuilder.matchInstanceListKey(datumJson.get("key").asText())) {
|
||||||
Datum<Instances> instancesDatum = new Datum<>();
|
Datum<Instances> instancesDatum = new Datum<>();
|
||||||
instancesDatum.key = datumJson.getString("key");
|
instancesDatum.key = datumJson.get("key").asText();
|
||||||
instancesDatum.timestamp.set(datumJson.getLongValue("timestamp"));
|
instancesDatum.timestamp.set(datumJson.get("timestamp").asLong());
|
||||||
instancesDatum.value =
|
instancesDatum.value = JacksonUtils.toObj(datumJson.get("value").toString(), Instances.class);
|
||||||
JSON.parseObject(JSON.toJSONString(datumJson.getJSONObject("value")), Instances.class);
|
|
||||||
newDatum = instancesDatum;
|
newDatum = instancesDatum;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -737,7 +736,7 @@ public class RaftCore {
|
|||||||
raftStore.updateTerm(local.term.get());
|
raftStore.updateTerm(local.term.get());
|
||||||
|
|
||||||
Loggers.RAFT.info("data updated, key: {}, timestamp: {}, from {}, local term: {}",
|
Loggers.RAFT.info("data updated, key: {}, timestamp: {}, from {}, local term: {}",
|
||||||
newDatum.key, newDatum.timestamp, JSON.toJSONString(remote), local.term);
|
newDatum.key, newDatum.timestamp, JacksonUtils.toJson(remote), local.term);
|
||||||
|
|
||||||
} catch (Throwable e) {
|
} catch (Throwable e) {
|
||||||
Loggers.RAFT.error("[RAFT-BEAT] failed to sync datum from leader, datum: {}", newDatum, e);
|
Loggers.RAFT.error("[RAFT-BEAT] failed to sync datum from leader, datum: {}", newDatum, e);
|
||||||
|
@ -16,7 +16,7 @@
|
|||||||
|
|
||||||
package com.alibaba.nacos.naming.consistency.persistent.raft;
|
package com.alibaba.nacos.naming.consistency.persistent.raft;
|
||||||
|
|
||||||
import com.alibaba.fastjson.JSON;
|
import com.alibaba.nacos.common.utils.JacksonUtils;
|
||||||
import com.alibaba.nacos.core.cluster.Member;
|
import com.alibaba.nacos.core.cluster.Member;
|
||||||
import com.alibaba.nacos.core.cluster.ServerMemberManager;
|
import com.alibaba.nacos.core.cluster.ServerMemberManager;
|
||||||
import org.springframework.beans.factory.annotation.Autowired;
|
import org.springframework.beans.factory.annotation.Autowired;
|
||||||
@ -52,8 +52,8 @@ public class RaftListener implements SmartApplicationListener {
|
|||||||
if (event instanceof BaseRaftEvent) {
|
if (event instanceof BaseRaftEvent) {
|
||||||
BaseRaftEvent raftEvent = (BaseRaftEvent) event;
|
BaseRaftEvent raftEvent = (BaseRaftEvent) event;
|
||||||
RaftPeer local = raftEvent.getLocal();
|
RaftPeer local = raftEvent.getLocal();
|
||||||
String json = JSON.toJSONString(local);
|
String json = JacksonUtils.toJson(local);
|
||||||
Map map = JSON.parseObject(json, HashMap.class);
|
Map map = JacksonUtils.toObj(json, HashMap.class);
|
||||||
Member self = memberManager.getSelf();
|
Member self = memberManager.getSelf();
|
||||||
self.setExtendVal(GROUP, map);
|
self.setExtendVal(GROUP, map);
|
||||||
memberManager.update(self);
|
memberManager.update(self);
|
||||||
|
@ -15,7 +15,7 @@
|
|||||||
*/
|
*/
|
||||||
package com.alibaba.nacos.naming.consistency.persistent.raft;
|
package com.alibaba.nacos.naming.consistency.persistent.raft;
|
||||||
|
|
||||||
import com.alibaba.fastjson.JSON;
|
import com.alibaba.nacos.common.utils.JacksonUtils;
|
||||||
import com.alibaba.nacos.core.cluster.Member;
|
import com.alibaba.nacos.core.cluster.Member;
|
||||||
import com.alibaba.nacos.core.cluster.MemberChangeListener;
|
import com.alibaba.nacos.core.cluster.MemberChangeListener;
|
||||||
import com.alibaba.nacos.core.cluster.MemberChangeEvent;
|
import com.alibaba.nacos.core.cluster.MemberChangeEvent;
|
||||||
@ -171,7 +171,7 @@ public class RaftPeerSet implements MemberChangeListener {
|
|||||||
leader = candidate;
|
leader = candidate;
|
||||||
ApplicationUtils.publishEvent(new MakeLeaderEvent(this, leader, local()));
|
ApplicationUtils.publishEvent(new MakeLeaderEvent(this, leader, local()));
|
||||||
Loggers.RAFT.info("{} has become the LEADER, local: {}, leader: {}",
|
Loggers.RAFT.info("{} has become the LEADER, local: {}, leader: {}",
|
||||||
leader.ip, JSON.toJSONString(local()), JSON.toJSONString(leader));
|
leader.ip, JacksonUtils.toJson(local()), JacksonUtils.toJson(leader));
|
||||||
}
|
}
|
||||||
|
|
||||||
for (final RaftPeer peer : peers.values()) {
|
for (final RaftPeer peer : peers.values()) {
|
||||||
@ -189,7 +189,7 @@ public class RaftPeerSet implements MemberChangeListener {
|
|||||||
return 1;
|
return 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
update(JSON.parseObject(response.getResponseBody(), RaftPeer.class));
|
update(JacksonUtils.toObj(response.getResponseBody(), RaftPeer.class));
|
||||||
|
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
@ -15,10 +15,8 @@
|
|||||||
*/
|
*/
|
||||||
package com.alibaba.nacos.naming.consistency.persistent.raft;
|
package com.alibaba.nacos.naming.consistency.persistent.raft;
|
||||||
|
|
||||||
import com.alibaba.fastjson.JSON;
|
|
||||||
import com.alibaba.fastjson.JSONObject;
|
|
||||||
import com.alibaba.fastjson.TypeReference;
|
|
||||||
import com.alibaba.nacos.api.common.Constants;
|
import com.alibaba.nacos.api.common.Constants;
|
||||||
|
import com.alibaba.nacos.common.utils.JacksonUtils;
|
||||||
import com.alibaba.nacos.naming.consistency.ApplyAction;
|
import com.alibaba.nacos.naming.consistency.ApplyAction;
|
||||||
import com.alibaba.nacos.naming.consistency.Datum;
|
import com.alibaba.nacos.naming.consistency.Datum;
|
||||||
import com.alibaba.nacos.naming.consistency.KeyBuilder;
|
import com.alibaba.nacos.naming.consistency.KeyBuilder;
|
||||||
@ -29,6 +27,9 @@ import com.alibaba.nacos.naming.misc.Loggers;
|
|||||||
import com.alibaba.nacos.naming.misc.SwitchDomain;
|
import com.alibaba.nacos.naming.misc.SwitchDomain;
|
||||||
import com.alibaba.nacos.naming.misc.UtilsAndCommons;
|
import com.alibaba.nacos.naming.misc.UtilsAndCommons;
|
||||||
import com.alibaba.nacos.naming.monitor.MetricsMonitor;
|
import com.alibaba.nacos.naming.monitor.MetricsMonitor;
|
||||||
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
|
import com.fasterxml.jackson.databind.JsonNode;
|
||||||
|
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.springframework.stereotype.Component;
|
import org.springframework.stereotype.Component;
|
||||||
|
|
||||||
@ -126,8 +127,7 @@ public class RaftStore {
|
|||||||
}
|
}
|
||||||
|
|
||||||
if (KeyBuilder.matchSwitchKey(file.getName())) {
|
if (KeyBuilder.matchSwitchKey(file.getName())) {
|
||||||
return JSON.parseObject(json, new TypeReference<Datum<SwitchDomain>>() {
|
return JacksonUtils.toObj(json, new TypeReference<Datum<SwitchDomain>>() {});
|
||||||
});
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if (KeyBuilder.matchServiceMetaKey(file.getName())) {
|
if (KeyBuilder.matchServiceMetaKey(file.getName())) {
|
||||||
@ -135,15 +135,14 @@ public class RaftStore {
|
|||||||
Datum<Service> serviceDatum;
|
Datum<Service> serviceDatum;
|
||||||
|
|
||||||
try {
|
try {
|
||||||
serviceDatum = JSON.parseObject(json.replace("\\", ""), new TypeReference<Datum<Service>>() {
|
serviceDatum = JacksonUtils.toObj(json.replace("\\", ""), new TypeReference<Datum<Service>>() {});
|
||||||
});
|
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
JSONObject jsonObject = JSON.parseObject(json);
|
JsonNode jsonObject = JacksonUtils.toObj(json);
|
||||||
|
|
||||||
serviceDatum = new Datum<>();
|
serviceDatum = new Datum<>();
|
||||||
serviceDatum.timestamp.set(jsonObject.getLongValue("timestamp"));
|
serviceDatum.timestamp.set(jsonObject.get("timestamp").asLong());
|
||||||
serviceDatum.key = jsonObject.getString("key");
|
serviceDatum.key = jsonObject.get("key").asText();
|
||||||
serviceDatum.value = JSON.parseObject(jsonObject.getString("value"), Service.class);
|
serviceDatum.value = JacksonUtils.toObj(jsonObject.get("value").toString(), Service.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (StringUtils.isBlank(serviceDatum.value.getGroupName())) {
|
if (StringUtils.isBlank(serviceDatum.value.getGroupName())) {
|
||||||
@ -162,23 +161,22 @@ public class RaftStore {
|
|||||||
Datum<Instances> instancesDatum;
|
Datum<Instances> instancesDatum;
|
||||||
|
|
||||||
try {
|
try {
|
||||||
instancesDatum = JSON.parseObject(json, new TypeReference<Datum<Instances>>() {
|
instancesDatum = JacksonUtils.toObj(json, new TypeReference<Datum<Instances>>() {});
|
||||||
});
|
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
JSONObject jsonObject = JSON.parseObject(json);
|
JsonNode jsonObject = JacksonUtils.toObj(json);
|
||||||
instancesDatum = new Datum<>();
|
instancesDatum = new Datum<>();
|
||||||
instancesDatum.timestamp.set(jsonObject.getLongValue("timestamp"));
|
instancesDatum.timestamp.set(jsonObject.get("timestamp").asLong());
|
||||||
|
|
||||||
String key = jsonObject.getString("key");
|
String key = jsonObject.get("key").asText();
|
||||||
String serviceName = KeyBuilder.getServiceName(key);
|
String serviceName = KeyBuilder.getServiceName(key);
|
||||||
key = key.substring(0, key.indexOf(serviceName)) +
|
key = key.substring(0, key.indexOf(serviceName)) +
|
||||||
Constants.DEFAULT_GROUP + Constants.SERVICE_INFO_SPLITER + serviceName;
|
Constants.DEFAULT_GROUP + Constants.SERVICE_INFO_SPLITER + serviceName;
|
||||||
|
|
||||||
instancesDatum.key = key;
|
instancesDatum.key = key;
|
||||||
instancesDatum.value = new Instances();
|
instancesDatum.value = new Instances();
|
||||||
instancesDatum.value.setInstanceList(JSON.parseObject(jsonObject.getString("value"),
|
instancesDatum.value.setInstanceList(JacksonUtils.toObj(jsonObject.get("value").toString(),
|
||||||
new TypeReference<List<Instance>>() {
|
new TypeReference<List<Instance>>() {})
|
||||||
}));
|
);
|
||||||
if (!instancesDatum.value.getInstanceList().isEmpty()) {
|
if (!instancesDatum.value.getInstanceList().isEmpty()) {
|
||||||
for (Instance instance : instancesDatum.value.getInstanceList()) {
|
for (Instance instance : instancesDatum.value.getInstanceList()) {
|
||||||
instance.setEphemeral(false);
|
instance.setEphemeral(false);
|
||||||
@ -189,7 +187,7 @@ public class RaftStore {
|
|||||||
return instancesDatum;
|
return instancesDatum;
|
||||||
}
|
}
|
||||||
|
|
||||||
return JSON.parseObject(json, Datum.class);
|
return JacksonUtils.toObj(json, Datum.class);
|
||||||
|
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
Loggers.RAFT.warn("waning: failed to deserialize key: {}", file.getName());
|
Loggers.RAFT.warn("waning: failed to deserialize key: {}", file.getName());
|
||||||
@ -222,7 +220,7 @@ public class RaftStore {
|
|||||||
FileChannel fc = null;
|
FileChannel fc = null;
|
||||||
ByteBuffer data;
|
ByteBuffer data;
|
||||||
|
|
||||||
data = ByteBuffer.wrap(JSON.toJSONString(datum).getBytes(StandardCharsets.UTF_8));
|
data = ByteBuffer.wrap(JacksonUtils.toJson(datum).getBytes(StandardCharsets.UTF_8));
|
||||||
|
|
||||||
try {
|
try {
|
||||||
fc = new FileOutputStream(cacheFile, false).getChannel();
|
fc = new FileOutputStream(cacheFile, false).getChannel();
|
||||||
|
@ -15,10 +15,10 @@
|
|||||||
*/
|
*/
|
||||||
package com.alibaba.nacos.naming.controllers;
|
package com.alibaba.nacos.naming.controllers;
|
||||||
|
|
||||||
import com.alibaba.fastjson.JSONObject;
|
|
||||||
import com.alibaba.nacos.api.common.Constants;
|
import com.alibaba.nacos.api.common.Constants;
|
||||||
import com.alibaba.nacos.api.naming.CommonParams;
|
import com.alibaba.nacos.api.naming.CommonParams;
|
||||||
import com.alibaba.nacos.api.naming.utils.NamingUtils;
|
import com.alibaba.nacos.api.naming.utils.NamingUtils;
|
||||||
|
import com.alibaba.nacos.common.utils.JacksonUtils;
|
||||||
import com.alibaba.nacos.core.utils.WebUtils;
|
import com.alibaba.nacos.core.utils.WebUtils;
|
||||||
import com.alibaba.nacos.naming.core.DistroMapper;
|
import com.alibaba.nacos.naming.core.DistroMapper;
|
||||||
import com.alibaba.nacos.naming.core.ServiceManager;
|
import com.alibaba.nacos.naming.core.ServiceManager;
|
||||||
@ -26,6 +26,8 @@ import com.alibaba.nacos.naming.misc.UtilsAndCommons;
|
|||||||
import com.alibaba.nacos.naming.push.ClientInfo;
|
import com.alibaba.nacos.naming.push.ClientInfo;
|
||||||
import com.alibaba.nacos.naming.web.CanDistro;
|
import com.alibaba.nacos.naming.web.CanDistro;
|
||||||
import com.alibaba.nacos.core.utils.OverrideParameterRequestWrapper;
|
import com.alibaba.nacos.core.utils.OverrideParameterRequestWrapper;
|
||||||
|
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||||
|
|
||||||
import org.apache.commons.collections.CollectionUtils;
|
import org.apache.commons.collections.CollectionUtils;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.codehaus.jackson.util.VersionUtil;
|
import org.codehaus.jackson.util.VersionUtil;
|
||||||
@ -54,11 +56,11 @@ public class ApiController extends InstanceController {
|
|||||||
private ServiceManager serviceManager;
|
private ServiceManager serviceManager;
|
||||||
|
|
||||||
@RequestMapping("/allDomNames")
|
@RequestMapping("/allDomNames")
|
||||||
public JSONObject allDomNames(HttpServletRequest request) throws Exception {
|
public ObjectNode allDomNames(HttpServletRequest request) throws Exception {
|
||||||
|
|
||||||
boolean responsibleOnly = Boolean.parseBoolean(WebUtils.optional(request, "responsibleOnly", "false"));
|
boolean responsibleOnly = Boolean.parseBoolean(WebUtils.optional(request, "responsibleOnly", "false"));
|
||||||
Map<String, Set<String>> domMap = serviceManager.getAllServiceNames();
|
Map<String, Set<String>> domMap = serviceManager.getAllServiceNames();
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
// For old DNS-F client:
|
// For old DNS-F client:
|
||||||
String dnsfVersion = "1.0.1";
|
String dnsfVersion = "1.0.1";
|
||||||
String agent = WebUtils.getUserAgent(request);
|
String agent = WebUtils.getUserAgent(request);
|
||||||
@ -74,7 +76,7 @@ public class ApiController extends InstanceController {
|
|||||||
}
|
}
|
||||||
|
|
||||||
if (CollectionUtils.isEmpty(domSet)) {
|
if (CollectionUtils.isEmpty(domSet)) {
|
||||||
result.put("doms", new HashSet<>());
|
result.put("doms", JacksonUtils.transferToJsonNode(new HashSet<>()));
|
||||||
result.put("count", 0);
|
result.put("count", 0);
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
@ -85,7 +87,7 @@ public class ApiController extends InstanceController {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
result.put("doms", doms);
|
result.put("doms", JacksonUtils.transferToJsonNode(doms));
|
||||||
result.put("count", doms.size());
|
result.put("count", doms.size());
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
@ -102,7 +104,7 @@ public class ApiController extends InstanceController {
|
|||||||
count += doms.get(namespaceId).size();
|
count += doms.get(namespaceId).size();
|
||||||
}
|
}
|
||||||
|
|
||||||
result.put("doms", doms);
|
result.put("doms", JacksonUtils.transferToJsonNode(doms));
|
||||||
result.put("count", count);
|
result.put("count", count);
|
||||||
|
|
||||||
return result;
|
return result;
|
||||||
@ -116,7 +118,7 @@ public class ApiController extends InstanceController {
|
|||||||
|
|
||||||
@RequestMapping("/srvIPXT")
|
@RequestMapping("/srvIPXT")
|
||||||
@ResponseBody
|
@ResponseBody
|
||||||
public JSONObject srvIPXT(HttpServletRequest request) throws Exception {
|
public ObjectNode srvIPXT(HttpServletRequest request) throws Exception {
|
||||||
|
|
||||||
String namespaceId = WebUtils.optional(request, CommonParams.NAMESPACE_ID,
|
String namespaceId = WebUtils.optional(request, CommonParams.NAMESPACE_ID,
|
||||||
Constants.DEFAULT_NAMESPACE_ID);
|
Constants.DEFAULT_NAMESPACE_ID);
|
||||||
@ -141,7 +143,7 @@ public class ApiController extends InstanceController {
|
|||||||
|
|
||||||
@CanDistro
|
@CanDistro
|
||||||
@RequestMapping("/clientBeat")
|
@RequestMapping("/clientBeat")
|
||||||
public JSONObject clientBeat(HttpServletRequest request) throws Exception {
|
public ObjectNode clientBeat(HttpServletRequest request) throws Exception {
|
||||||
OverrideParameterRequestWrapper requestWrapper = OverrideParameterRequestWrapper.buildRequest(request);
|
OverrideParameterRequestWrapper requestWrapper = OverrideParameterRequestWrapper.buildRequest(request);
|
||||||
requestWrapper.addParameter(CommonParams.SERVICE_NAME,
|
requestWrapper.addParameter(CommonParams.SERVICE_NAME,
|
||||||
Constants.DEFAULT_GROUP + Constants.SERVICE_INFO_SPLITER + WebUtils.required(request, "dom"));
|
Constants.DEFAULT_GROUP + Constants.SERVICE_INFO_SPLITER + WebUtils.required(request, "dom"));
|
||||||
|
@ -15,13 +15,12 @@
|
|||||||
*/
|
*/
|
||||||
package com.alibaba.nacos.naming.controllers;
|
package com.alibaba.nacos.naming.controllers;
|
||||||
|
|
||||||
import com.alibaba.fastjson.JSONArray;
|
|
||||||
import com.alibaba.fastjson.JSONObject;
|
|
||||||
import com.alibaba.nacos.api.common.Constants;
|
import com.alibaba.nacos.api.common.Constants;
|
||||||
import com.alibaba.nacos.api.exception.NacosException;
|
import com.alibaba.nacos.api.exception.NacosException;
|
||||||
import com.alibaba.nacos.api.naming.CommonParams;
|
import com.alibaba.nacos.api.naming.CommonParams;
|
||||||
import com.alibaba.nacos.api.naming.pojo.Cluster;
|
import com.alibaba.nacos.api.naming.pojo.Cluster;
|
||||||
import com.alibaba.nacos.api.naming.utils.NamingUtils;
|
import com.alibaba.nacos.api.naming.utils.NamingUtils;
|
||||||
|
import com.alibaba.nacos.common.utils.JacksonUtils;
|
||||||
import com.alibaba.nacos.core.auth.ActionTypes;
|
import com.alibaba.nacos.core.auth.ActionTypes;
|
||||||
import com.alibaba.nacos.core.auth.Secured;
|
import com.alibaba.nacos.core.auth.Secured;
|
||||||
import com.alibaba.nacos.core.utils.WebUtils;
|
import com.alibaba.nacos.core.utils.WebUtils;
|
||||||
@ -35,6 +34,9 @@ import com.alibaba.nacos.naming.pojo.IpAddressInfo;
|
|||||||
import com.alibaba.nacos.naming.pojo.ServiceDetailInfo;
|
import com.alibaba.nacos.naming.pojo.ServiceDetailInfo;
|
||||||
import com.alibaba.nacos.naming.pojo.ServiceView;
|
import com.alibaba.nacos.naming.pojo.ServiceView;
|
||||||
import com.alibaba.nacos.naming.web.NamingResourceParser;
|
import com.alibaba.nacos.naming.web.NamingResourceParser;
|
||||||
|
import com.fasterxml.jackson.databind.node.ArrayNode;
|
||||||
|
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||||
|
|
||||||
import org.apache.commons.collections.CollectionUtils;
|
import org.apache.commons.collections.CollectionUtils;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.springframework.beans.factory.annotation.Autowired;
|
import org.springframework.beans.factory.annotation.Autowired;
|
||||||
@ -58,7 +60,7 @@ public class CatalogController {
|
|||||||
|
|
||||||
@Secured(parser = NamingResourceParser.class, action = ActionTypes.READ)
|
@Secured(parser = NamingResourceParser.class, action = ActionTypes.READ)
|
||||||
@GetMapping("/service")
|
@GetMapping("/service")
|
||||||
public JSONObject serviceDetail(@RequestParam(defaultValue = Constants.DEFAULT_NAMESPACE_ID) String namespaceId,
|
public ObjectNode serviceDetail(@RequestParam(defaultValue = Constants.DEFAULT_NAMESPACE_ID) String namespaceId,
|
||||||
String serviceName) throws NacosException {
|
String serviceName) throws NacosException {
|
||||||
|
|
||||||
Service detailedService = serviceManager.getService(namespaceId, serviceName);
|
Service detailedService = serviceManager.getService(namespaceId, serviceName);
|
||||||
@ -67,16 +69,16 @@ public class CatalogController {
|
|||||||
throw new NacosException(NacosException.NOT_FOUND, "service " + serviceName + " is not found!");
|
throw new NacosException(NacosException.NOT_FOUND, "service " + serviceName + " is not found!");
|
||||||
}
|
}
|
||||||
|
|
||||||
JSONObject detailView = new JSONObject();
|
ObjectNode detailView = JacksonUtils.createEmptyJsonNode();
|
||||||
|
|
||||||
JSONObject serviceObject = new JSONObject();
|
ObjectNode serviceObject = JacksonUtils.createEmptyJsonNode();
|
||||||
serviceObject.put("name", NamingUtils.getServiceName(serviceName));
|
serviceObject.put("name", NamingUtils.getServiceName(serviceName));
|
||||||
serviceObject.put("protectThreshold", detailedService.getProtectThreshold());
|
serviceObject.put("protectThreshold", detailedService.getProtectThreshold());
|
||||||
serviceObject.put("groupName", NamingUtils.getGroupName(serviceName));
|
serviceObject.put("groupName", NamingUtils.getGroupName(serviceName));
|
||||||
serviceObject.put("selector", detailedService.getSelector());
|
serviceObject.replace("selector", JacksonUtils.transferToJsonNode(detailedService.getSelector()));
|
||||||
serviceObject.put("metadata", detailedService.getMetadata());
|
serviceObject.replace("metadata", JacksonUtils.transferToJsonNode(detailedService.getMetadata()));
|
||||||
|
|
||||||
detailView.put("service", serviceObject);
|
detailView.replace("service", serviceObject);
|
||||||
|
|
||||||
List<Cluster> clusters = new ArrayList<>();
|
List<Cluster> clusters = new ArrayList<>();
|
||||||
|
|
||||||
@ -92,14 +94,14 @@ public class CatalogController {
|
|||||||
clusters.add(clusterView);
|
clusters.add(clusterView);
|
||||||
}
|
}
|
||||||
|
|
||||||
detailView.put("clusters", clusters);
|
detailView.replace("clusters", JacksonUtils.transferToJsonNode(clusters));
|
||||||
|
|
||||||
return detailView;
|
return detailView;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Secured(parser = NamingResourceParser.class, action = ActionTypes.READ)
|
@Secured(parser = NamingResourceParser.class, action = ActionTypes.READ)
|
||||||
@RequestMapping(value = "/instances")
|
@RequestMapping(value = "/instances")
|
||||||
public JSONObject instanceList(@RequestParam(defaultValue = Constants.DEFAULT_NAMESPACE_ID) String namespaceId,
|
public ObjectNode instanceList(@RequestParam(defaultValue = Constants.DEFAULT_NAMESPACE_ID) String namespaceId,
|
||||||
@RequestParam String serviceName,
|
@RequestParam String serviceName,
|
||||||
@RequestParam String clusterName,
|
@RequestParam String clusterName,
|
||||||
@RequestParam(name = "pageNo") int page,
|
@RequestParam(name = "pageNo") int page,
|
||||||
@ -132,8 +134,8 @@ public class CatalogController {
|
|||||||
end = instances.size();
|
end = instances.size();
|
||||||
}
|
}
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
result.put("list", instances.subList(start, end));
|
result.replace("list", JacksonUtils.transferToJsonNode(instances.subList(start, end)));
|
||||||
result.put("count", instances.size());
|
result.put("count", instances.size());
|
||||||
|
|
||||||
return result;
|
return result;
|
||||||
@ -175,19 +177,19 @@ public class CatalogController {
|
|||||||
return serviceDetailInfoList;
|
return serviceDetailInfoList;
|
||||||
}
|
}
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
|
|
||||||
List<Service> services = new ArrayList<>();
|
List<Service> services = new ArrayList<>();
|
||||||
|
|
||||||
int total = serviceManager.getPagedService(namespaceId, pageNo - 1, pageSize, param, containedInstance, services, hasIpCount);
|
int total = serviceManager.getPagedService(namespaceId, pageNo - 1, pageSize, param, containedInstance, services, hasIpCount);
|
||||||
|
|
||||||
if (CollectionUtils.isEmpty(services)) {
|
if (CollectionUtils.isEmpty(services)) {
|
||||||
result.put("serviceList", Collections.emptyList());
|
result.replace("serviceList", JacksonUtils.transferToJsonNode(Collections.emptyList()));
|
||||||
result.put("count", 0);
|
result.put("count", 0);
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
JSONArray serviceJsonArray = new JSONArray();
|
List<ServiceView> serviceViews = new LinkedList<>();
|
||||||
for (Service service : services) {
|
for (Service service : services) {
|
||||||
ServiceView serviceView = new ServiceView();
|
ServiceView serviceView = new ServiceView();
|
||||||
serviceView.setName(NamingUtils.getServiceName(service.getName()));
|
serviceView.setName(NamingUtils.getServiceName(service.getName()));
|
||||||
@ -196,17 +198,17 @@ public class CatalogController {
|
|||||||
serviceView.setIpCount(service.allIPs().size());
|
serviceView.setIpCount(service.allIPs().size());
|
||||||
serviceView.setHealthyInstanceCount(service.healthyInstanceCount());
|
serviceView.setHealthyInstanceCount(service.healthyInstanceCount());
|
||||||
serviceView.setTriggerFlag(service.triggerFlag() ? "true" : "false");
|
serviceView.setTriggerFlag(service.triggerFlag() ? "true" : "false");
|
||||||
serviceJsonArray.add(serviceView);
|
serviceViews.add(serviceView);
|
||||||
}
|
}
|
||||||
|
|
||||||
result.put("serviceList", serviceJsonArray);
|
result.replace("serviceList", JacksonUtils.transferToJsonNode(serviceViews));
|
||||||
result.put("count", total);
|
result.put("count", total);
|
||||||
|
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
@RequestMapping("/rt/service")
|
@RequestMapping("/rt/service")
|
||||||
public JSONObject rt4Service(HttpServletRequest request) {
|
public ObjectNode rt4Service(HttpServletRequest request) {
|
||||||
|
|
||||||
String namespaceId = WebUtils.optional(request, CommonParams.NAMESPACE_ID,
|
String namespaceId = WebUtils.optional(request, CommonParams.NAMESPACE_ID,
|
||||||
Constants.DEFAULT_NAMESPACE_ID);
|
Constants.DEFAULT_NAMESPACE_ID);
|
||||||
@ -218,11 +220,11 @@ public class CatalogController {
|
|||||||
throw new IllegalArgumentException("request service doesn't exist");
|
throw new IllegalArgumentException("request service doesn't exist");
|
||||||
}
|
}
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
|
|
||||||
JSONArray clusters = new JSONArray();
|
ArrayNode clusters = JacksonUtils.createEmptyArrayNode();
|
||||||
for (Map.Entry<String, com.alibaba.nacos.naming.core.Cluster> entry : service.getClusterMap().entrySet()) {
|
for (Map.Entry<String, com.alibaba.nacos.naming.core.Cluster> entry : service.getClusterMap().entrySet()) {
|
||||||
JSONObject packet = new JSONObject();
|
ObjectNode packet = JacksonUtils.createEmptyJsonNode();
|
||||||
HealthCheckTask task = entry.getValue().getHealthCheckTask();
|
HealthCheckTask task = entry.getValue().getHealthCheckTask();
|
||||||
|
|
||||||
packet.put("name", entry.getKey());
|
packet.put("name", entry.getKey());
|
||||||
@ -232,8 +234,7 @@ public class CatalogController {
|
|||||||
|
|
||||||
clusters.add(packet);
|
clusters.add(packet);
|
||||||
}
|
}
|
||||||
result.put("clusters", clusters);
|
result.replace("clusters", clusters);
|
||||||
|
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -15,8 +15,8 @@
|
|||||||
*/
|
*/
|
||||||
package com.alibaba.nacos.naming.controllers;
|
package com.alibaba.nacos.naming.controllers;
|
||||||
|
|
||||||
import com.alibaba.fastjson.JSONObject;
|
|
||||||
import com.alibaba.nacos.api.exception.NacosException;
|
import com.alibaba.nacos.api.exception.NacosException;
|
||||||
|
import com.alibaba.nacos.common.utils.JacksonUtils;
|
||||||
import com.alibaba.nacos.naming.cluster.transport.Serializer;
|
import com.alibaba.nacos.naming.cluster.transport.Serializer;
|
||||||
import com.alibaba.nacos.naming.consistency.Datum;
|
import com.alibaba.nacos.naming.consistency.Datum;
|
||||||
import com.alibaba.nacos.naming.consistency.KeyBuilder;
|
import com.alibaba.nacos.naming.consistency.KeyBuilder;
|
||||||
@ -27,6 +27,8 @@ import com.alibaba.nacos.naming.core.ServiceManager;
|
|||||||
import com.alibaba.nacos.naming.misc.Loggers;
|
import com.alibaba.nacos.naming.misc.Loggers;
|
||||||
import com.alibaba.nacos.naming.misc.SwitchDomain;
|
import com.alibaba.nacos.naming.misc.SwitchDomain;
|
||||||
import com.alibaba.nacos.naming.misc.UtilsAndCommons;
|
import com.alibaba.nacos.naming.misc.UtilsAndCommons;
|
||||||
|
import com.fasterxml.jackson.databind.JsonNode;
|
||||||
|
|
||||||
import org.springframework.beans.factory.annotation.Autowired;
|
import org.springframework.beans.factory.annotation.Autowired;
|
||||||
import org.springframework.http.ResponseEntity;
|
import org.springframework.http.ResponseEntity;
|
||||||
import org.springframework.web.bind.annotation.*;
|
import org.springframework.web.bind.annotation.*;
|
||||||
@ -90,9 +92,10 @@ public class DistroController {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@GetMapping("/datum")
|
@GetMapping("/datum")
|
||||||
public ResponseEntity get(@RequestBody JSONObject body) throws Exception {
|
public ResponseEntity get(@RequestBody String body) throws Exception {
|
||||||
|
|
||||||
String keys = body.getString("keys");
|
JsonNode bodyNode = JacksonUtils.toObj(body);
|
||||||
|
String keys = bodyNode.get("keys").asText();
|
||||||
String keySplitter = ",";
|
String keySplitter = ",";
|
||||||
Map<String, Datum> datumMap = new HashMap<>(64);
|
Map<String, Datum> datumMap = new HashMap<>(64);
|
||||||
for (String key : keys.split(keySplitter)) {
|
for (String key : keys.split(keySplitter)) {
|
||||||
|
@ -15,10 +15,10 @@
|
|||||||
*/
|
*/
|
||||||
package com.alibaba.nacos.naming.controllers;
|
package com.alibaba.nacos.naming.controllers;
|
||||||
|
|
||||||
import com.alibaba.fastjson.JSONObject;
|
|
||||||
import com.alibaba.nacos.api.common.Constants;
|
import com.alibaba.nacos.api.common.Constants;
|
||||||
import com.alibaba.nacos.api.naming.CommonParams;
|
import com.alibaba.nacos.api.naming.CommonParams;
|
||||||
import com.alibaba.nacos.api.naming.pojo.healthcheck.AbstractHealthChecker;
|
import com.alibaba.nacos.api.naming.pojo.healthcheck.AbstractHealthChecker;
|
||||||
|
import com.alibaba.nacos.common.utils.JacksonUtils;
|
||||||
import com.alibaba.nacos.core.auth.ActionTypes;
|
import com.alibaba.nacos.core.auth.ActionTypes;
|
||||||
import com.alibaba.nacos.core.auth.Secured;
|
import com.alibaba.nacos.core.auth.Secured;
|
||||||
import com.alibaba.nacos.core.utils.ApplicationUtils;
|
import com.alibaba.nacos.core.utils.ApplicationUtils;
|
||||||
@ -31,6 +31,7 @@ import com.alibaba.nacos.naming.misc.Loggers;
|
|||||||
import com.alibaba.nacos.naming.misc.UtilsAndCommons;
|
import com.alibaba.nacos.naming.misc.UtilsAndCommons;
|
||||||
import com.alibaba.nacos.naming.push.PushService;
|
import com.alibaba.nacos.naming.push.PushService;
|
||||||
import com.alibaba.nacos.naming.web.CanDistro;
|
import com.alibaba.nacos.naming.web.CanDistro;
|
||||||
|
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import org.apache.commons.lang3.BooleanUtils;
|
import org.apache.commons.lang3.BooleanUtils;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
@ -65,8 +66,8 @@ public class HealthController {
|
|||||||
private PushService pushService;
|
private PushService pushService;
|
||||||
|
|
||||||
@RequestMapping("/server")
|
@RequestMapping("/server")
|
||||||
public JSONObject server() {
|
public ObjectNode server() {
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
result.put("msg", "Hello! I am Nacos-Naming and healthy! total services: raft " + serviceManager.getServiceCount()
|
result.put("msg", "Hello! I am Nacos-Naming and healthy! total services: raft " + serviceManager.getServiceCount()
|
||||||
+ ", local port:" + ApplicationUtils.getPort());
|
+ ", local port:" + ApplicationUtils.getPort());
|
||||||
return result;
|
return result;
|
||||||
|
@ -15,14 +15,12 @@
|
|||||||
*/
|
*/
|
||||||
package com.alibaba.nacos.naming.controllers;
|
package com.alibaba.nacos.naming.controllers;
|
||||||
|
|
||||||
import com.alibaba.fastjson.JSON;
|
|
||||||
import com.alibaba.fastjson.JSONArray;
|
|
||||||
import com.alibaba.fastjson.JSONObject;
|
|
||||||
import com.alibaba.nacos.api.common.Constants;
|
import com.alibaba.nacos.api.common.Constants;
|
||||||
import com.alibaba.nacos.api.exception.NacosException;
|
import com.alibaba.nacos.api.exception.NacosException;
|
||||||
import com.alibaba.nacos.api.naming.CommonParams;
|
import com.alibaba.nacos.api.naming.CommonParams;
|
||||||
import com.alibaba.nacos.api.naming.NamingResponseCode;
|
import com.alibaba.nacos.api.naming.NamingResponseCode;
|
||||||
import com.alibaba.nacos.api.naming.utils.NamingUtils;
|
import com.alibaba.nacos.api.naming.utils.NamingUtils;
|
||||||
|
import com.alibaba.nacos.common.utils.JacksonUtils;
|
||||||
import com.alibaba.nacos.core.auth.ActionTypes;
|
import com.alibaba.nacos.core.auth.ActionTypes;
|
||||||
import com.alibaba.nacos.core.auth.Secured;
|
import com.alibaba.nacos.core.auth.Secured;
|
||||||
import com.alibaba.nacos.core.utils.WebUtils;
|
import com.alibaba.nacos.core.utils.WebUtils;
|
||||||
@ -39,6 +37,9 @@ import com.alibaba.nacos.naming.push.DataSource;
|
|||||||
import com.alibaba.nacos.naming.push.PushService;
|
import com.alibaba.nacos.naming.push.PushService;
|
||||||
import com.alibaba.nacos.naming.web.CanDistro;
|
import com.alibaba.nacos.naming.web.CanDistro;
|
||||||
import com.alibaba.nacos.naming.web.NamingResourceParser;
|
import com.alibaba.nacos.naming.web.NamingResourceParser;
|
||||||
|
import com.fasterxml.jackson.databind.node.ArrayNode;
|
||||||
|
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||||
|
|
||||||
import org.apache.commons.collections.CollectionUtils;
|
import org.apache.commons.collections.CollectionUtils;
|
||||||
import org.apache.commons.lang3.BooleanUtils;
|
import org.apache.commons.lang3.BooleanUtils;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
@ -84,7 +85,7 @@ public class InstanceController {
|
|||||||
@Override
|
@Override
|
||||||
public String getData(PushService.PushClient client) {
|
public String getData(PushService.PushClient client) {
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
try {
|
try {
|
||||||
result = doSrvIPXT(client.getNamespaceId(), client.getServiceName(),
|
result = doSrvIPXT(client.getNamespaceId(), client.getServiceName(),
|
||||||
client.getAgent(), client.getClusters(),
|
client.getAgent(), client.getClusters(),
|
||||||
@ -100,7 +101,7 @@ public class InstanceController {
|
|||||||
result.put("cacheMillis",
|
result.put("cacheMillis",
|
||||||
switchDomain.getPushCacheMillis(client.getServiceName()));
|
switchDomain.getPushCacheMillis(client.getServiceName()));
|
||||||
|
|
||||||
return result.toJSONString();
|
return result.toString();
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -214,7 +215,7 @@ public class InstanceController {
|
|||||||
|
|
||||||
@GetMapping("/list")
|
@GetMapping("/list")
|
||||||
@Secured(parser = NamingResourceParser.class, action = ActionTypes.READ)
|
@Secured(parser = NamingResourceParser.class, action = ActionTypes.READ)
|
||||||
public JSONObject list(HttpServletRequest request) throws Exception {
|
public ObjectNode list(HttpServletRequest request) throws Exception {
|
||||||
|
|
||||||
String namespaceId = WebUtils.optional(request, CommonParams.NAMESPACE_ID,
|
String namespaceId = WebUtils.optional(request, CommonParams.NAMESPACE_ID,
|
||||||
Constants.DEFAULT_NAMESPACE_ID);
|
Constants.DEFAULT_NAMESPACE_ID);
|
||||||
@ -241,7 +242,7 @@ public class InstanceController {
|
|||||||
|
|
||||||
@GetMapping
|
@GetMapping
|
||||||
@Secured(parser = NamingResourceParser.class, action = ActionTypes.READ)
|
@Secured(parser = NamingResourceParser.class, action = ActionTypes.READ)
|
||||||
public JSONObject detail(HttpServletRequest request) throws Exception {
|
public ObjectNode detail(HttpServletRequest request) throws Exception {
|
||||||
|
|
||||||
String namespaceId = WebUtils.optional(request, CommonParams.NAMESPACE_ID,
|
String namespaceId = WebUtils.optional(request, CommonParams.NAMESPACE_ID,
|
||||||
Constants.DEFAULT_NAMESPACE_ID);
|
Constants.DEFAULT_NAMESPACE_ID);
|
||||||
@ -268,14 +269,14 @@ public class InstanceController {
|
|||||||
|
|
||||||
for (Instance instance : ips) {
|
for (Instance instance : ips) {
|
||||||
if (instance.getIp().equals(ip) && instance.getPort() == port) {
|
if (instance.getIp().equals(ip) && instance.getPort() == port) {
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
result.put("service", serviceName);
|
result.put("service", serviceName);
|
||||||
result.put("ip", ip);
|
result.put("ip", ip);
|
||||||
result.put("port", port);
|
result.put("port", port);
|
||||||
result.put("clusterName", cluster);
|
result.put("clusterName", cluster);
|
||||||
result.put("weight", instance.getWeight());
|
result.put("weight", instance.getWeight());
|
||||||
result.put("healthy", instance.isHealthy());
|
result.put("healthy", instance.isHealthy());
|
||||||
result.put("metadata", instance.getMetadata());
|
result.put("metadata", JacksonUtils.transferToJsonNode(instance.getMetadata()));
|
||||||
result.put("instanceId", instance.getInstanceId());
|
result.put("instanceId", instance.getInstanceId());
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
@ -287,9 +288,9 @@ public class InstanceController {
|
|||||||
@CanDistro
|
@CanDistro
|
||||||
@PutMapping("/beat")
|
@PutMapping("/beat")
|
||||||
@Secured(parser = NamingResourceParser.class, action = ActionTypes.WRITE)
|
@Secured(parser = NamingResourceParser.class, action = ActionTypes.WRITE)
|
||||||
public JSONObject beat(HttpServletRequest request) throws Exception {
|
public ObjectNode beat(HttpServletRequest request) throws Exception {
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
|
|
||||||
result.put("clientBeatInterval", switchDomain.getClientBeatInterval());
|
result.put("clientBeatInterval", switchDomain.getClientBeatInterval());
|
||||||
String serviceName = WebUtils.required(request, CommonParams.SERVICE_NAME);
|
String serviceName = WebUtils.required(request, CommonParams.SERVICE_NAME);
|
||||||
@ -303,7 +304,7 @@ public class InstanceController {
|
|||||||
|
|
||||||
RsInfo clientBeat = null;
|
RsInfo clientBeat = null;
|
||||||
if (StringUtils.isNotBlank(beat)) {
|
if (StringUtils.isNotBlank(beat)) {
|
||||||
clientBeat = JSON.parseObject(beat, RsInfo.class);
|
clientBeat = JacksonUtils.toObj(beat, RsInfo.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (clientBeat != null) {
|
if (clientBeat != null) {
|
||||||
@ -368,7 +369,7 @@ public class InstanceController {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@RequestMapping("/statuses")
|
@RequestMapping("/statuses")
|
||||||
public JSONObject listWithHealthStatus(@RequestParam String key)
|
public ObjectNode listWithHealthStatus(@RequestParam String key)
|
||||||
throws NacosException {
|
throws NacosException {
|
||||||
|
|
||||||
String serviceName;
|
String serviceName;
|
||||||
@ -392,14 +393,14 @@ public class InstanceController {
|
|||||||
|
|
||||||
List<Instance> ips = service.allIPs();
|
List<Instance> ips = service.allIPs();
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
JSONArray ipArray = new JSONArray();
|
ArrayNode ipArray = JacksonUtils.createEmptyArrayNode();
|
||||||
|
|
||||||
for (Instance ip : ips) {
|
for (Instance ip : ips) {
|
||||||
ipArray.add(ip.toIPAddr() + "_" + ip.isHealthy());
|
ipArray.add(ip.toIPAddr() + "_" + ip.isHealthy());
|
||||||
}
|
}
|
||||||
|
|
||||||
result.put("ips", ipArray);
|
result.replace("ips", ipArray);
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -470,12 +471,12 @@ public class InstanceController {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public JSONObject doSrvIPXT(String namespaceId, String serviceName, String agent,
|
public ObjectNode doSrvIPXT(String namespaceId, String serviceName, String agent,
|
||||||
String clusters, String clientIP, int udpPort, String env, boolean isCheck,
|
String clusters, String clientIP, int udpPort, String env, boolean isCheck,
|
||||||
String app, String tid, boolean healthyOnly) throws Exception {
|
String app, String tid, boolean healthyOnly) throws Exception {
|
||||||
|
|
||||||
ClientInfo clientInfo = new ClientInfo(agent);
|
ClientInfo clientInfo = new ClientInfo(agent);
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
Service service = serviceManager.getService(namespaceId, serviceName);
|
Service service = serviceManager.getService(namespaceId, serviceName);
|
||||||
|
|
||||||
if (service == null) {
|
if (service == null) {
|
||||||
@ -485,7 +486,7 @@ public class InstanceController {
|
|||||||
}
|
}
|
||||||
result.put("name", serviceName);
|
result.put("name", serviceName);
|
||||||
result.put("clusters", clusters);
|
result.put("clusters", clusters);
|
||||||
result.put("hosts", new JSONArray());
|
result.replace("hosts", JacksonUtils.createEmptyArrayNode());
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -534,7 +535,7 @@ public class InstanceController {
|
|||||||
result.put("dom", NamingUtils.getServiceName(serviceName));
|
result.put("dom", NamingUtils.getServiceName(serviceName));
|
||||||
}
|
}
|
||||||
|
|
||||||
result.put("hosts", new JSONArray());
|
result.put("hosts", JacksonUtils.createEmptyArrayNode());
|
||||||
result.put("name", serviceName);
|
result.put("name", serviceName);
|
||||||
result.put("cacheMillis", cacheMillis);
|
result.put("cacheMillis", cacheMillis);
|
||||||
result.put("lastRefTime", System.currentTimeMillis());
|
result.put("lastRefTime", System.currentTimeMillis());
|
||||||
@ -542,7 +543,7 @@ public class InstanceController {
|
|||||||
result.put("useSpecifiedURL", false);
|
result.put("useSpecifiedURL", false);
|
||||||
result.put("clusters", clusters);
|
result.put("clusters", clusters);
|
||||||
result.put("env", env);
|
result.put("env", env);
|
||||||
result.put("metadata", service.getMetadata());
|
result.put("metadata", JacksonUtils.transferToJsonNode(service.getMetadata()));
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -576,10 +577,10 @@ public class InstanceController {
|
|||||||
result.put("protectThreshold", service.getProtectThreshold());
|
result.put("protectThreshold", service.getProtectThreshold());
|
||||||
result.put("reachLocalSiteCallThreshold", false);
|
result.put("reachLocalSiteCallThreshold", false);
|
||||||
|
|
||||||
return new JSONObject();
|
return JacksonUtils.createEmptyJsonNode();
|
||||||
}
|
}
|
||||||
|
|
||||||
JSONArray hosts = new JSONArray();
|
ArrayNode hosts = JacksonUtils.createEmptyArrayNode();
|
||||||
|
|
||||||
for (Map.Entry<Boolean, List<Instance>> entry : ipMap.entrySet()) {
|
for (Map.Entry<Boolean, List<Instance>> entry : ipMap.entrySet()) {
|
||||||
List<Instance> ips = entry.getValue();
|
List<Instance> ips = entry.getValue();
|
||||||
@ -595,7 +596,7 @@ public class InstanceController {
|
|||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
JSONObject ipObj = new JSONObject();
|
ObjectNode ipObj = JacksonUtils.createEmptyJsonNode();
|
||||||
|
|
||||||
ipObj.put("ip", instance.getIp());
|
ipObj.put("ip", instance.getIp());
|
||||||
ipObj.put("port", instance.getPort());
|
ipObj.put("port", instance.getPort());
|
||||||
@ -604,7 +605,7 @@ public class InstanceController {
|
|||||||
ipObj.put("healthy", entry.getKey());
|
ipObj.put("healthy", entry.getKey());
|
||||||
ipObj.put("marked", instance.isMarked());
|
ipObj.put("marked", instance.isMarked());
|
||||||
ipObj.put("instanceId", instance.getInstanceId());
|
ipObj.put("instanceId", instance.getInstanceId());
|
||||||
ipObj.put("metadata", instance.getMetadata());
|
ipObj.put("metadata", JacksonUtils.transferToJsonNode(instance.getMetadata()));
|
||||||
ipObj.put("enabled", instance.isEnabled());
|
ipObj.put("enabled", instance.isEnabled());
|
||||||
ipObj.put("weight", instance.getWeight());
|
ipObj.put("weight", instance.getWeight());
|
||||||
ipObj.put("clusterName", instance.getClusterName());
|
ipObj.put("clusterName", instance.getClusterName());
|
||||||
@ -624,7 +625,7 @@ public class InstanceController {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
result.put("hosts", hosts);
|
result.replace("hosts", hosts);
|
||||||
if (clientInfo.type == ClientInfo.ClientType.JAVA
|
if (clientInfo.type == ClientInfo.ClientType.JAVA
|
||||||
&& clientInfo.version.compareTo(VersionUtil.parseVersion("1.0.0")) >= 0) {
|
&& clientInfo.version.compareTo(VersionUtil.parseVersion("1.0.0")) >= 0) {
|
||||||
result.put("dom", serviceName);
|
result.put("dom", serviceName);
|
||||||
@ -639,7 +640,7 @@ public class InstanceController {
|
|||||||
result.put("useSpecifiedURL", false);
|
result.put("useSpecifiedURL", false);
|
||||||
result.put("clusters", clusters);
|
result.put("clusters", clusters);
|
||||||
result.put("env", env);
|
result.put("env", env);
|
||||||
result.put("metadata", service.getMetadata());
|
result.replace("metadata", JacksonUtils.transferToJsonNode(service.getMetadata()));
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -15,12 +15,11 @@
|
|||||||
*/
|
*/
|
||||||
package com.alibaba.nacos.naming.controllers;
|
package com.alibaba.nacos.naming.controllers;
|
||||||
|
|
||||||
import com.alibaba.fastjson.JSON;
|
|
||||||
import com.alibaba.fastjson.JSONArray;
|
|
||||||
import com.alibaba.fastjson.JSONObject;
|
|
||||||
import com.alibaba.nacos.api.common.Constants;
|
import com.alibaba.nacos.api.common.Constants;
|
||||||
|
import com.alibaba.nacos.common.utils.JacksonUtils;
|
||||||
import com.alibaba.nacos.core.auth.ActionTypes;
|
import com.alibaba.nacos.core.auth.ActionTypes;
|
||||||
import com.alibaba.nacos.core.auth.Secured;
|
import com.alibaba.nacos.core.auth.Secured;
|
||||||
|
import com.alibaba.nacos.core.cluster.Member;
|
||||||
import com.alibaba.nacos.core.cluster.NodeState;
|
import com.alibaba.nacos.core.cluster.NodeState;
|
||||||
import com.alibaba.nacos.core.cluster.ServerMemberManager;
|
import com.alibaba.nacos.core.cluster.ServerMemberManager;
|
||||||
import com.alibaba.nacos.core.utils.ApplicationUtils;
|
import com.alibaba.nacos.core.utils.ApplicationUtils;
|
||||||
@ -32,6 +31,10 @@ import com.alibaba.nacos.naming.core.Service;
|
|||||||
import com.alibaba.nacos.naming.core.ServiceManager;
|
import com.alibaba.nacos.naming.core.ServiceManager;
|
||||||
import com.alibaba.nacos.naming.misc.*;
|
import com.alibaba.nacos.naming.misc.*;
|
||||||
import com.alibaba.nacos.naming.push.PushService;
|
import com.alibaba.nacos.naming.push.PushService;
|
||||||
|
import com.fasterxml.jackson.databind.JsonNode;
|
||||||
|
import com.fasterxml.jackson.databind.node.ArrayNode;
|
||||||
|
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||||
|
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.springframework.beans.factory.annotation.Autowired;
|
import org.springframework.beans.factory.annotation.Autowired;
|
||||||
import org.springframework.web.bind.annotation.*;
|
import org.springframework.web.bind.annotation.*;
|
||||||
@ -75,9 +78,9 @@ public class OperatorController {
|
|||||||
private RaftCore raftCore;
|
private RaftCore raftCore;
|
||||||
|
|
||||||
@RequestMapping("/push/state")
|
@RequestMapping("/push/state")
|
||||||
public JSONObject pushState(@RequestParam(required = false) boolean detail, @RequestParam(required = false) boolean reset) {
|
public ObjectNode pushState(@RequestParam(required = false) boolean detail, @RequestParam(required = false) boolean reset) {
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
|
|
||||||
List<PushService.Receiver.AckEntry> failedPushes = PushService.getFailedPushes();
|
List<PushService.Receiver.AckEntry> failedPushes = PushService.getFailedPushes();
|
||||||
int failedPushCount = pushService.getFailedPushCount();
|
int failedPushCount = pushService.getFailedPushCount();
|
||||||
@ -90,7 +93,7 @@ public class OperatorController {
|
|||||||
result.put("ratio", 0);
|
result.put("ratio", 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
JSONArray dataArray = new JSONArray();
|
ArrayNode dataArray = JacksonUtils.createEmptyArrayNode();
|
||||||
if (detail) {
|
if (detail) {
|
||||||
for (PushService.Receiver.AckEntry entry : failedPushes) {
|
for (PushService.Receiver.AckEntry entry : failedPushes) {
|
||||||
try {
|
try {
|
||||||
@ -99,7 +102,7 @@ public class OperatorController {
|
|||||||
dataArray.add("[encoding failure]");
|
dataArray.add("[encoding failure]");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
result.put("data", dataArray);
|
result.replace("data", dataArray);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (reset) {
|
if (reset) {
|
||||||
@ -128,9 +131,9 @@ public class OperatorController {
|
|||||||
|
|
||||||
@Secured(resource = "naming/metrics", action = ActionTypes.READ)
|
@Secured(resource = "naming/metrics", action = ActionTypes.READ)
|
||||||
@GetMapping("/metrics")
|
@GetMapping("/metrics")
|
||||||
public JSONObject metrics(HttpServletRequest request) {
|
public ObjectNode metrics(HttpServletRequest request) {
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
|
|
||||||
int serviceCount = serviceManager.getServiceCount();
|
int serviceCount = serviceManager.getServiceCount();
|
||||||
int ipCount = serviceManager.getInstanceCount();
|
int ipCount = serviceManager.getInstanceCount();
|
||||||
@ -152,7 +155,7 @@ public class OperatorController {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@GetMapping("/distro/server")
|
@GetMapping("/distro/server")
|
||||||
public JSONObject getResponsibleServer4Service(@RequestParam(defaultValue = Constants.DEFAULT_NAMESPACE_ID) String namespaceId,
|
public ObjectNode getResponsibleServer4Service(@RequestParam(defaultValue = Constants.DEFAULT_NAMESPACE_ID) String namespaceId,
|
||||||
@RequestParam String serviceName) {
|
@RequestParam String serviceName) {
|
||||||
|
|
||||||
Service service = serviceManager.getService(namespaceId, serviceName);
|
Service service = serviceManager.getService(namespaceId, serviceName);
|
||||||
@ -161,7 +164,7 @@ public class OperatorController {
|
|||||||
throw new IllegalArgumentException("service not found");
|
throw new IllegalArgumentException("service not found");
|
||||||
}
|
}
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
|
|
||||||
result.put("responsibleServer", distroMapper.mapSrv(serviceName));
|
result.put("responsibleServer", distroMapper.mapSrv(serviceName));
|
||||||
|
|
||||||
@ -169,12 +172,12 @@ public class OperatorController {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@GetMapping("/distro/status")
|
@GetMapping("/distro/status")
|
||||||
public JSONObject distroStatus(@RequestParam(defaultValue = "view") String action) {
|
public ObjectNode distroStatus(@RequestParam(defaultValue = "view") String action) {
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
|
|
||||||
if (StringUtils.equals(SwitchEntry.ACTION_VIEW, action)) {
|
if (StringUtils.equals(SwitchEntry.ACTION_VIEW, action)) {
|
||||||
result.put("status", memberManager.allMembers());
|
result.replace("status", JacksonUtils.transferToJsonNode(memberManager.allMembers()));
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -182,15 +185,16 @@ public class OperatorController {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@GetMapping("/servers")
|
@GetMapping("/servers")
|
||||||
public JSONObject getHealthyServerList(@RequestParam(required = false) boolean healthy) {
|
public ObjectNode getHealthyServerList(@RequestParam(required = false) boolean healthy) {
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
if (healthy) {
|
if (healthy) {
|
||||||
result.put("servers", memberManager.allMembers().stream()
|
List<Member> healthyMember = memberManager.allMembers().stream()
|
||||||
.filter(member -> member.getState() == NodeState.UP).collect(ArrayList::new,
|
.filter(member -> member.getState() == NodeState.UP).collect(ArrayList::new,
|
||||||
ArrayList::add, ArrayList::addAll));
|
ArrayList::add, ArrayList::addAll);
|
||||||
|
result.replace("servers", JacksonUtils.transferToJsonNode(healthyMember));
|
||||||
} else {
|
} else {
|
||||||
result.put("servers", memberManager.allMembers());
|
result.replace("servers", JacksonUtils.transferToJsonNode(memberManager.allMembers()));
|
||||||
}
|
}
|
||||||
|
|
||||||
return result;
|
return result;
|
||||||
@ -203,11 +207,7 @@ public class OperatorController {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@RequestMapping(value = "/cluster/state", method = RequestMethod.GET)
|
@RequestMapping(value = "/cluster/state", method = RequestMethod.GET)
|
||||||
public JSONObject getClusterStates() {
|
public JsonNode getClusterStates() {
|
||||||
|
return JacksonUtils.transferToJsonNode(serviceManager.getMySelfClusterState());
|
||||||
RaftPeer peer = serviceManager.getMySelfClusterState();
|
|
||||||
|
|
||||||
return JSON.parseObject(JSON.toJSONString(peer));
|
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -15,12 +15,9 @@
|
|||||||
*/
|
*/
|
||||||
package com.alibaba.nacos.naming.controllers;
|
package com.alibaba.nacos.naming.controllers;
|
||||||
|
|
||||||
import com.alibaba.fastjson.JSON;
|
|
||||||
import com.alibaba.fastjson.JSONArray;
|
|
||||||
import com.alibaba.fastjson.JSONObject;
|
|
||||||
import com.alibaba.fastjson.TypeReference;
|
|
||||||
import com.alibaba.nacos.api.exception.NacosException;
|
import com.alibaba.nacos.api.exception.NacosException;
|
||||||
import com.alibaba.nacos.common.utils.IoUtils;
|
import com.alibaba.nacos.common.utils.IoUtils;
|
||||||
|
import com.alibaba.nacos.common.utils.JacksonUtils;
|
||||||
import com.alibaba.nacos.core.utils.WebUtils;
|
import com.alibaba.nacos.core.utils.WebUtils;
|
||||||
import com.alibaba.nacos.naming.consistency.Datum;
|
import com.alibaba.nacos.naming.consistency.Datum;
|
||||||
import com.alibaba.nacos.naming.consistency.KeyBuilder;
|
import com.alibaba.nacos.naming.consistency.KeyBuilder;
|
||||||
@ -34,6 +31,11 @@ import com.alibaba.nacos.naming.core.ServiceManager;
|
|||||||
import com.alibaba.nacos.naming.misc.NetUtils;
|
import com.alibaba.nacos.naming.misc.NetUtils;
|
||||||
import com.alibaba.nacos.naming.misc.SwitchDomain;
|
import com.alibaba.nacos.naming.misc.SwitchDomain;
|
||||||
import com.alibaba.nacos.naming.misc.UtilsAndCommons;
|
import com.alibaba.nacos.naming.misc.UtilsAndCommons;
|
||||||
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
|
import com.fasterxml.jackson.databind.JsonNode;
|
||||||
|
import com.fasterxml.jackson.databind.node.ArrayNode;
|
||||||
|
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||||
|
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.springframework.beans.factory.annotation.Autowired;
|
import org.springframework.beans.factory.annotation.Autowired;
|
||||||
import org.springframework.web.bind.annotation.*;
|
import org.springframework.web.bind.annotation.*;
|
||||||
@ -67,31 +69,30 @@ public class RaftController {
|
|||||||
private RaftCore raftCore;
|
private RaftCore raftCore;
|
||||||
|
|
||||||
@PostMapping("/vote")
|
@PostMapping("/vote")
|
||||||
public JSONObject vote(HttpServletRequest request, HttpServletResponse response) throws Exception {
|
public JsonNode vote(HttpServletRequest request, HttpServletResponse response) throws Exception {
|
||||||
|
|
||||||
RaftPeer peer = raftCore.receivedVote(
|
RaftPeer peer = raftCore.receivedVote(
|
||||||
JSON.parseObject(WebUtils.required(request, "vote"), RaftPeer.class));
|
JacksonUtils.toObj(WebUtils.required(request, "vote"), RaftPeer.class));
|
||||||
|
|
||||||
return JSON.parseObject(JSON.toJSONString(peer));
|
return JacksonUtils.transferToJsonNode(peer);
|
||||||
}
|
}
|
||||||
|
|
||||||
@PostMapping("/beat")
|
@PostMapping("/beat")
|
||||||
public JSONObject beat(HttpServletRequest request, HttpServletResponse response) throws Exception {
|
public JsonNode beat(HttpServletRequest request, HttpServletResponse response) throws Exception {
|
||||||
|
|
||||||
String entity = new String(IoUtils.tryDecompress(request.getInputStream()), StandardCharsets.UTF_8);
|
String entity = new String(IoUtils.tryDecompress(request.getInputStream()), StandardCharsets.UTF_8);
|
||||||
String value = URLDecoder.decode(entity, "UTF-8");
|
String value = URLDecoder.decode(entity, "UTF-8");
|
||||||
value = URLDecoder.decode(value, "UTF-8");
|
value = URLDecoder.decode(value, "UTF-8");
|
||||||
|
|
||||||
JSONObject json = JSON.parseObject(value);
|
JsonNode json = JacksonUtils.toObj(value);
|
||||||
JSONObject beat = JSON.parseObject(json.getString("beat"));
|
|
||||||
|
|
||||||
RaftPeer peer = raftCore.receivedBeat(beat);
|
RaftPeer peer = raftCore.receivedBeat(json.get("beat"));
|
||||||
|
|
||||||
return JSON.parseObject(JSON.toJSONString(peer));
|
return JacksonUtils.transferToJsonNode(peer);
|
||||||
}
|
}
|
||||||
|
|
||||||
@GetMapping("/peer")
|
@GetMapping("/peer")
|
||||||
public JSONObject getPeer(HttpServletRequest request, HttpServletResponse response) {
|
public JsonNode getPeer(HttpServletRequest request, HttpServletResponse response) {
|
||||||
List<RaftPeer> peers = raftCore.getPeers();
|
List<RaftPeer> peers = raftCore.getPeers();
|
||||||
RaftPeer peer = null;
|
RaftPeer peer = null;
|
||||||
|
|
||||||
@ -106,7 +107,7 @@ public class RaftController {
|
|||||||
peer.ip = NetUtils.localServer();
|
peer.ip = NetUtils.localServer();
|
||||||
}
|
}
|
||||||
|
|
||||||
return JSON.parseObject(JSON.toJSONString(peer));
|
return JacksonUtils.transferToJsonNode(peer);
|
||||||
}
|
}
|
||||||
|
|
||||||
@PutMapping("/datum/reload")
|
@PutMapping("/datum/reload")
|
||||||
@ -125,21 +126,21 @@ public class RaftController {
|
|||||||
|
|
||||||
String entity = IoUtils.toString(request.getInputStream(), "UTF-8");
|
String entity = IoUtils.toString(request.getInputStream(), "UTF-8");
|
||||||
String value = URLDecoder.decode(entity, "UTF-8");
|
String value = URLDecoder.decode(entity, "UTF-8");
|
||||||
JSONObject json = JSON.parseObject(value);
|
JsonNode json = JacksonUtils.toObj(value);
|
||||||
|
|
||||||
String key = json.getString("key");
|
String key = json.get("key").toString();
|
||||||
if (KeyBuilder.matchInstanceListKey(key)) {
|
if (KeyBuilder.matchInstanceListKey(key)) {
|
||||||
raftConsistencyService.put(key, JSON.parseObject(json.getString("value"), Instances.class));
|
raftConsistencyService.put(key, JacksonUtils.toObj(json.get("value").toString(), Instances.class));
|
||||||
return "ok";
|
return "ok";
|
||||||
}
|
}
|
||||||
|
|
||||||
if (KeyBuilder.matchSwitchKey(key)) {
|
if (KeyBuilder.matchSwitchKey(key)) {
|
||||||
raftConsistencyService.put(key, JSON.parseObject(json.getString("value"), SwitchDomain.class));
|
raftConsistencyService.put(key, JacksonUtils.toObj(json.get("value").toString(), SwitchDomain.class));
|
||||||
return "ok";
|
return "ok";
|
||||||
}
|
}
|
||||||
|
|
||||||
if (KeyBuilder.matchServiceMetaKey(key)) {
|
if (KeyBuilder.matchServiceMetaKey(key)) {
|
||||||
raftConsistencyService.put(key, JSON.parseObject(json.getString("value"), Service.class));
|
raftConsistencyService.put(key, JacksonUtils.toObj(json.get("value").toString(), Service.class));
|
||||||
return "ok";
|
return "ok";
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -172,19 +173,19 @@ public class RaftController {
|
|||||||
datums.add(datum);
|
datums.add(datum);
|
||||||
}
|
}
|
||||||
|
|
||||||
return JSON.toJSONString(datums);
|
return JacksonUtils.toJson(datums);
|
||||||
}
|
}
|
||||||
|
|
||||||
@GetMapping("/state")
|
@GetMapping("/state")
|
||||||
public JSONObject state(HttpServletRequest request, HttpServletResponse response) throws Exception {
|
public JsonNode state(HttpServletRequest request, HttpServletResponse response) throws Exception {
|
||||||
|
|
||||||
response.setHeader("Content-Type", "application/json; charset=" + getAcceptEncoding(request));
|
response.setHeader("Content-Type", "application/json; charset=" + getAcceptEncoding(request));
|
||||||
response.setHeader("Cache-Control", "no-cache");
|
response.setHeader("Cache-Control", "no-cache");
|
||||||
response.setHeader("Content-Encode", "gzip");
|
response.setHeader("Content-Encode", "gzip");
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
result.put("services", serviceManager.getServiceCount());
|
result.put("services", serviceManager.getServiceCount());
|
||||||
result.put("peers", raftCore.getPeers());
|
result.replace("peers", JacksonUtils.transferToJsonNode(raftCore.getPeers()));
|
||||||
|
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
@ -198,21 +199,22 @@ public class RaftController {
|
|||||||
|
|
||||||
String entity = IoUtils.toString(request.getInputStream(), "UTF-8");
|
String entity = IoUtils.toString(request.getInputStream(), "UTF-8");
|
||||||
String value = URLDecoder.decode(entity, "UTF-8");
|
String value = URLDecoder.decode(entity, "UTF-8");
|
||||||
JSONObject jsonObject = JSON.parseObject(value);
|
|
||||||
|
JsonNode jsonObject = JacksonUtils.toObj(value);
|
||||||
String key = "key";
|
String key = "key";
|
||||||
|
|
||||||
RaftPeer source = JSON.parseObject(jsonObject.getString("source"), RaftPeer.class);
|
RaftPeer source = JacksonUtils.toObj(jsonObject.get("source").toString(), RaftPeer.class);
|
||||||
JSONObject datumJson = jsonObject.getJSONObject("datum");
|
JsonNode datumJson = jsonObject.get("datum");
|
||||||
|
|
||||||
Datum datum = null;
|
Datum datum = null;
|
||||||
if (KeyBuilder.matchInstanceListKey(datumJson.getString(key))) {
|
if (KeyBuilder.matchInstanceListKey(datumJson.get(key).asText())) {
|
||||||
datum = JSON.parseObject(jsonObject.getString("datum"), new TypeReference<Datum<Instances>>() {
|
datum = JacksonUtils.toObj(jsonObject.get("datum").toString(), new TypeReference<Datum<Instances>>() {
|
||||||
});
|
});
|
||||||
} else if (KeyBuilder.matchSwitchKey(datumJson.getString(key))) {
|
} else if (KeyBuilder.matchSwitchKey(datumJson.get(key).asText())) {
|
||||||
datum = JSON.parseObject(jsonObject.getString("datum"), new TypeReference<Datum<SwitchDomain>>() {
|
datum = JacksonUtils.toObj(jsonObject.get("datum").toString(), new TypeReference<Datum<SwitchDomain>>() {
|
||||||
});
|
});
|
||||||
} else if (KeyBuilder.matchServiceMetaKey(datumJson.getString(key))) {
|
} else if (KeyBuilder.matchServiceMetaKey(datumJson.get(key).asText())) {
|
||||||
datum = JSON.parseObject(jsonObject.getString("datum"), new TypeReference<Datum<Service>>() {
|
datum = JacksonUtils.toObj(jsonObject.get("datum").toString(), new TypeReference<Datum<Service>>() {
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -230,34 +232,35 @@ public class RaftController {
|
|||||||
String entity = IoUtils.toString(request.getInputStream(), "UTF-8");
|
String entity = IoUtils.toString(request.getInputStream(), "UTF-8");
|
||||||
String value = URLDecoder.decode(entity, "UTF-8");
|
String value = URLDecoder.decode(entity, "UTF-8");
|
||||||
value = URLDecoder.decode(value, "UTF-8");
|
value = URLDecoder.decode(value, "UTF-8");
|
||||||
JSONObject jsonObject = JSON.parseObject(value);
|
|
||||||
|
|
||||||
Datum datum = JSON.parseObject(jsonObject.getString("datum"), Datum.class);
|
JsonNode jsonObject = JacksonUtils.createEmptyJsonNode();
|
||||||
RaftPeer source = JSON.parseObject(jsonObject.getString("source"), RaftPeer.class);
|
|
||||||
|
Datum datum = JacksonUtils.toObj(jsonObject.get("datum").toString(), Datum.class);
|
||||||
|
RaftPeer source = JacksonUtils.toObj(jsonObject.get("source").toString(), RaftPeer.class);
|
||||||
|
|
||||||
raftConsistencyService.onRemove(datum, source);
|
raftConsistencyService.onRemove(datum, source);
|
||||||
return "ok";
|
return "ok";
|
||||||
}
|
}
|
||||||
|
|
||||||
@GetMapping("/leader")
|
@GetMapping("/leader")
|
||||||
public JSONObject getLeader(HttpServletRequest request, HttpServletResponse response) {
|
public JsonNode getLeader(HttpServletRequest request, HttpServletResponse response) {
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
result.put("leader", JSONObject.toJSONString(raftCore.getLeader()));
|
result.put("leader", JacksonUtils.toJson(raftCore.getLeader()));
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
@GetMapping("/listeners")
|
@GetMapping("/listeners")
|
||||||
public JSONObject getAllListeners(HttpServletRequest request, HttpServletResponse response) {
|
public JsonNode getAllListeners(HttpServletRequest request, HttpServletResponse response) {
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
Map<String, List<RecordListener>> listeners = raftCore.getListeners();
|
Map<String, List<RecordListener>> listeners = raftCore.getListeners();
|
||||||
|
|
||||||
JSONArray listenerArray = new JSONArray();
|
ArrayNode listenerArray = JacksonUtils.createEmptyArrayNode();
|
||||||
for (String key : listeners.keySet()) {
|
for (String key : listeners.keySet()) {
|
||||||
listenerArray.add(key);
|
listenerArray.add(key);
|
||||||
}
|
}
|
||||||
result.put("listeners", listenerArray);
|
result.replace("listeners", listenerArray);
|
||||||
|
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
@ -15,15 +15,13 @@
|
|||||||
*/
|
*/
|
||||||
package com.alibaba.nacos.naming.controllers;
|
package com.alibaba.nacos.naming.controllers;
|
||||||
|
|
||||||
import com.alibaba.fastjson.JSON;
|
|
||||||
import com.alibaba.fastjson.JSONArray;
|
|
||||||
import com.alibaba.fastjson.JSONObject;
|
|
||||||
import com.alibaba.nacos.api.common.Constants;
|
import com.alibaba.nacos.api.common.Constants;
|
||||||
import com.alibaba.nacos.api.exception.NacosException;
|
import com.alibaba.nacos.api.exception.NacosException;
|
||||||
import com.alibaba.nacos.api.naming.CommonParams;
|
import com.alibaba.nacos.api.naming.CommonParams;
|
||||||
import com.alibaba.nacos.api.naming.utils.NamingUtils;
|
import com.alibaba.nacos.api.naming.utils.NamingUtils;
|
||||||
import com.alibaba.nacos.api.selector.SelectorType;
|
import com.alibaba.nacos.api.selector.SelectorType;
|
||||||
import com.alibaba.nacos.common.utils.IoUtils;
|
import com.alibaba.nacos.common.utils.IoUtils;
|
||||||
|
import com.alibaba.nacos.common.utils.JacksonUtils;
|
||||||
import com.alibaba.nacos.core.auth.ActionTypes;
|
import com.alibaba.nacos.core.auth.ActionTypes;
|
||||||
import com.alibaba.nacos.core.auth.Secured;
|
import com.alibaba.nacos.core.auth.Secured;
|
||||||
import com.alibaba.nacos.core.cluster.ServerMemberManager;
|
import com.alibaba.nacos.core.cluster.ServerMemberManager;
|
||||||
@ -36,6 +34,10 @@ import com.alibaba.nacos.naming.selector.LabelSelector;
|
|||||||
import com.alibaba.nacos.naming.selector.NoneSelector;
|
import com.alibaba.nacos.naming.selector.NoneSelector;
|
||||||
import com.alibaba.nacos.naming.selector.Selector;
|
import com.alibaba.nacos.naming.selector.Selector;
|
||||||
import com.alibaba.nacos.naming.web.NamingResourceParser;
|
import com.alibaba.nacos.naming.web.NamingResourceParser;
|
||||||
|
import com.fasterxml.jackson.databind.JsonNode;
|
||||||
|
import com.fasterxml.jackson.databind.node.ArrayNode;
|
||||||
|
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||||
|
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.apache.commons.lang3.math.NumberUtils;
|
import org.apache.commons.lang3.math.NumberUtils;
|
||||||
import org.springframework.beans.factory.annotation.Autowired;
|
import org.springframework.beans.factory.annotation.Autowired;
|
||||||
@ -113,7 +115,7 @@ public class ServiceController {
|
|||||||
|
|
||||||
@GetMapping
|
@GetMapping
|
||||||
@Secured(parser = NamingResourceParser.class, action = ActionTypes.READ)
|
@Secured(parser = NamingResourceParser.class, action = ActionTypes.READ)
|
||||||
public JSONObject detail(@RequestParam(defaultValue = Constants.DEFAULT_NAMESPACE_ID) String namespaceId,
|
public ObjectNode detail(@RequestParam(defaultValue = Constants.DEFAULT_NAMESPACE_ID) String namespaceId,
|
||||||
@RequestParam String serviceName) throws NacosException {
|
@RequestParam String serviceName) throws NacosException {
|
||||||
|
|
||||||
Service service = serviceManager.getService(namespaceId, serviceName);
|
Service service = serviceManager.getService(namespaceId, serviceName);
|
||||||
@ -121,31 +123,31 @@ public class ServiceController {
|
|||||||
throw new NacosException(NacosException.NOT_FOUND, "service " + serviceName + " is not found!");
|
throw new NacosException(NacosException.NOT_FOUND, "service " + serviceName + " is not found!");
|
||||||
}
|
}
|
||||||
|
|
||||||
JSONObject res = new JSONObject();
|
ObjectNode res = JacksonUtils.createEmptyJsonNode();
|
||||||
res.put("name", NamingUtils.getServiceName(serviceName));
|
res.put("name", NamingUtils.getServiceName(serviceName));
|
||||||
res.put("namespaceId", service.getNamespaceId());
|
res.put("namespaceId", service.getNamespaceId());
|
||||||
res.put("protectThreshold", service.getProtectThreshold());
|
res.put("protectThreshold", service.getProtectThreshold());
|
||||||
res.put("metadata", service.getMetadata());
|
res.replace("metadata", JacksonUtils.transferToJsonNode(service.getMetadata()));
|
||||||
res.put("selector", service.getSelector());
|
res.replace("selector", JacksonUtils.transferToJsonNode(service.getSelector()));
|
||||||
res.put("groupName", NamingUtils.getGroupName(serviceName));
|
res.put("groupName", NamingUtils.getGroupName(serviceName));
|
||||||
|
|
||||||
JSONArray clusters = new JSONArray();
|
ArrayNode clusters = JacksonUtils.createEmptyArrayNode();
|
||||||
for (Cluster cluster : service.getClusterMap().values()) {
|
for (Cluster cluster : service.getClusterMap().values()) {
|
||||||
JSONObject clusterJson = new JSONObject();
|
ObjectNode clusterJson = JacksonUtils.createEmptyJsonNode();
|
||||||
clusterJson.put("name", cluster.getName());
|
clusterJson.put("name", cluster.getName());
|
||||||
clusterJson.put("healthChecker", cluster.getHealthChecker());
|
clusterJson.replace("healthChecker", JacksonUtils.transferToJsonNode(cluster.getHealthChecker()));
|
||||||
clusterJson.put("metadata", cluster.getMetadata());
|
clusterJson.replace("metadata", JacksonUtils.transferToJsonNode(cluster.getMetadata()));
|
||||||
clusters.add(clusterJson);
|
clusters.add(clusterJson);
|
||||||
}
|
}
|
||||||
|
|
||||||
res.put("clusters", clusters);
|
res.replace("clusters", clusters);
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
@GetMapping("/list")
|
@GetMapping("/list")
|
||||||
@Secured(parser = NamingResourceParser.class, action = ActionTypes.READ)
|
@Secured(parser = NamingResourceParser.class, action = ActionTypes.READ)
|
||||||
public JSONObject list(HttpServletRequest request) throws Exception {
|
public ObjectNode list(HttpServletRequest request) throws Exception {
|
||||||
|
|
||||||
int pageNo = NumberUtils.toInt(WebUtils.required(request, "pageNo"));
|
int pageNo = NumberUtils.toInt(WebUtils.required(request, "pageNo"));
|
||||||
int pageSize = NumberUtils.toInt(WebUtils.required(request, "pageSize"));
|
int pageSize = NumberUtils.toInt(WebUtils.required(request, "pageSize"));
|
||||||
@ -156,10 +158,10 @@ public class ServiceController {
|
|||||||
|
|
||||||
List<String> serviceNameList = serviceManager.getAllServiceNameList(namespaceId);
|
List<String> serviceNameList = serviceManager.getAllServiceNameList(namespaceId);
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
|
|
||||||
if (serviceNameList == null || serviceNameList.isEmpty()) {
|
if (serviceNameList == null || serviceNameList.isEmpty()) {
|
||||||
result.put("doms", Collections.emptyList());
|
result.replace("doms", JacksonUtils.transferToJsonNode(Collections.emptyList()));
|
||||||
result.put("count", 0);
|
result.put("count", 0);
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
@ -168,10 +170,10 @@ public class ServiceController {
|
|||||||
|
|
||||||
if (StringUtils.isNotBlank(selectorString)) {
|
if (StringUtils.isNotBlank(selectorString)) {
|
||||||
|
|
||||||
JSONObject selectorJson = JSON.parseObject(selectorString);
|
JsonNode selectorJson = JacksonUtils.toObj(selectorString);
|
||||||
|
|
||||||
SelectorType selectorType = SelectorType.valueOf(selectorJson.getString("type"));
|
SelectorType selectorType = SelectorType.valueOf(selectorJson.get("type").asText());
|
||||||
String expression = selectorJson.getString("expression");
|
String expression = selectorJson.get("expression").asText();
|
||||||
|
|
||||||
if (SelectorType.label.equals(selectorType) && StringUtils.isNotBlank(expression)) {
|
if (SelectorType.label.equals(selectorType) && StringUtils.isNotBlank(expression)) {
|
||||||
expression = StringUtils.deleteWhitespace(expression);
|
expression = StringUtils.deleteWhitespace(expression);
|
||||||
@ -208,7 +210,7 @@ public class ServiceController {
|
|||||||
serviceNameList.set(i, serviceNameList.get(i).replace(groupName + Constants.SERVICE_INFO_SPLITER, ""));
|
serviceNameList.set(i, serviceNameList.get(i).replace(groupName + Constants.SERVICE_INFO_SPLITER, ""));
|
||||||
}
|
}
|
||||||
|
|
||||||
result.put("doms", serviceNameList.subList(start, end));
|
result.replace("doms", JacksonUtils.transferToJsonNode(serviceNameList.subList(start, end)));
|
||||||
result.put("count", serviceNameList.size());
|
result.put("count", serviceNameList.size());
|
||||||
|
|
||||||
return result;
|
return result;
|
||||||
@ -247,7 +249,7 @@ public class ServiceController {
|
|||||||
|
|
||||||
@RequestMapping("/names")
|
@RequestMapping("/names")
|
||||||
@Secured(parser = NamingResourceParser.class, action = ActionTypes.READ)
|
@Secured(parser = NamingResourceParser.class, action = ActionTypes.READ)
|
||||||
public JSONObject searchService(@RequestParam(defaultValue = StringUtils.EMPTY) String namespaceId,
|
public ObjectNode searchService(@RequestParam(defaultValue = StringUtils.EMPTY) String namespaceId,
|
||||||
@RequestParam(defaultValue = StringUtils.EMPTY) String expr,
|
@RequestParam(defaultValue = StringUtils.EMPTY) String expr,
|
||||||
@RequestParam(required = false) boolean responsibleOnly) {
|
@RequestParam(required = false) boolean responsibleOnly) {
|
||||||
|
|
||||||
@ -270,9 +272,9 @@ public class ServiceController {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
|
|
||||||
result.put("services", serviceNameMap);
|
result.replace("services", JacksonUtils.transferToJsonNode(serviceNameMap));
|
||||||
result.put("count", services.size());
|
result.put("count", services.size());
|
||||||
|
|
||||||
return result;
|
return result;
|
||||||
@ -283,11 +285,11 @@ public class ServiceController {
|
|||||||
|
|
||||||
String entity = IoUtils.toString(request.getInputStream(), "UTF-8");
|
String entity = IoUtils.toString(request.getInputStream(), "UTF-8");
|
||||||
String value = URLDecoder.decode(entity, "UTF-8");
|
String value = URLDecoder.decode(entity, "UTF-8");
|
||||||
JSONObject json = JSON.parseObject(value);
|
JsonNode json = JacksonUtils.toObj(value);
|
||||||
|
|
||||||
//format: service1@@checksum@@@service2@@checksum
|
//format: service1@@checksum@@@service2@@checksum
|
||||||
String statuses = json.getString("statuses");
|
String statuses = json.get("statuses").toString();
|
||||||
String serverIp = json.getString("clientIP");
|
String serverIp = json.get("clientIP").toString();
|
||||||
|
|
||||||
if (!memberManager.hasMember(serverIp)) {
|
if (!memberManager.hasMember(serverIp)) {
|
||||||
throw new NacosException(NacosException.INVALID_PARAM,
|
throw new NacosException(NacosException.INVALID_PARAM,
|
||||||
@ -295,7 +297,7 @@ public class ServiceController {
|
|||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
ServiceManager.ServiceChecksum checksums = JSON.parseObject(statuses, ServiceManager.ServiceChecksum.class);
|
ServiceManager.ServiceChecksum checksums = JacksonUtils.toObj(statuses, ServiceManager.ServiceChecksum.class);
|
||||||
if (checksums == null) {
|
if (checksums == null) {
|
||||||
Loggers.SRV_LOG.warn("[DOMAIN-STATUS] receive malformed data: null");
|
Loggers.SRV_LOG.warn("[DOMAIN-STATUS] receive malformed data: null");
|
||||||
return "fail";
|
return "fail";
|
||||||
@ -331,7 +333,7 @@ public class ServiceController {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@PutMapping("/checksum")
|
@PutMapping("/checksum")
|
||||||
public JSONObject checksum(HttpServletRequest request) throws Exception {
|
public ObjectNode checksum(HttpServletRequest request) throws Exception {
|
||||||
|
|
||||||
String namespaceId = WebUtils.optional(request, CommonParams.NAMESPACE_ID,
|
String namespaceId = WebUtils.optional(request, CommonParams.NAMESPACE_ID,
|
||||||
Constants.DEFAULT_NAMESPACE_ID);
|
Constants.DEFAULT_NAMESPACE_ID);
|
||||||
@ -345,7 +347,7 @@ public class ServiceController {
|
|||||||
|
|
||||||
service.recalculateChecksum();
|
service.recalculateChecksum();
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
|
|
||||||
result.put("checksum", service.getChecksum());
|
result.put("checksum", service.getChecksum());
|
||||||
|
|
||||||
@ -360,7 +362,7 @@ public class ServiceController {
|
|||||||
*/
|
*/
|
||||||
@GetMapping("/subscribers")
|
@GetMapping("/subscribers")
|
||||||
@Secured(parser = NamingResourceParser.class, action = ActionTypes.READ)
|
@Secured(parser = NamingResourceParser.class, action = ActionTypes.READ)
|
||||||
public JSONObject subscribers(HttpServletRequest request) {
|
public ObjectNode subscribers(HttpServletRequest request) {
|
||||||
|
|
||||||
int pageNo = NumberUtils.toInt(WebUtils.required(request, "pageNo"));
|
int pageNo = NumberUtils.toInt(WebUtils.required(request, "pageNo"));
|
||||||
int pageSize = NumberUtils.toInt(WebUtils.required(request, "pageSize"));
|
int pageSize = NumberUtils.toInt(WebUtils.required(request, "pageSize"));
|
||||||
@ -370,7 +372,7 @@ public class ServiceController {
|
|||||||
String serviceName = WebUtils.required(request, CommonParams.SERVICE_NAME);
|
String serviceName = WebUtils.required(request, CommonParams.SERVICE_NAME);
|
||||||
boolean aggregation = Boolean.parseBoolean(WebUtils.optional(request, "aggregation", String.valueOf(Boolean.TRUE)));
|
boolean aggregation = Boolean.parseBoolean(WebUtils.optional(request, "aggregation", String.valueOf(Boolean.TRUE)));
|
||||||
|
|
||||||
JSONObject result = new JSONObject();
|
ObjectNode result = JacksonUtils.createEmptyJsonNode();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
List<Subscriber> subscribers = subscribeManager.getSubscribers(serviceName, namespaceId, aggregation);
|
List<Subscriber> subscribers = subscribeManager.getSubscribers(serviceName, namespaceId, aggregation);
|
||||||
@ -388,13 +390,13 @@ public class ServiceController {
|
|||||||
end = count;
|
end = count;
|
||||||
}
|
}
|
||||||
|
|
||||||
result.put("subscribers", subscribers.subList(start, end));
|
result.replace("subscribers", JacksonUtils.transferToJsonNode(subscribers.subList(start, end)));
|
||||||
result.put("count", count);
|
result.put("count", count);
|
||||||
|
|
||||||
return result;
|
return result;
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
Loggers.SRV_LOG.warn("query subscribers failed!", e);
|
Loggers.SRV_LOG.warn("query subscribers failed!", e);
|
||||||
result.put("subscribers", new JSONArray());
|
result.replace("subscribers", JacksonUtils.createEmptyArrayNode());
|
||||||
result.put("count", 0);
|
result.put("count", 0);
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
@ -440,12 +442,12 @@ public class ServiceController {
|
|||||||
return new NoneSelector();
|
return new NoneSelector();
|
||||||
}
|
}
|
||||||
|
|
||||||
JSONObject selectorJson = JSON.parseObject(URLDecoder.decode(selectorJsonString, "UTF-8"));
|
JsonNode selectorJson = JacksonUtils.toObj(URLDecoder.decode(selectorJsonString, "UTF-8"));
|
||||||
switch (SelectorType.valueOf(selectorJson.getString("type"))) {
|
switch (SelectorType.valueOf(selectorJson.get("type").asText())) {
|
||||||
case none:
|
case none:
|
||||||
return new NoneSelector();
|
return new NoneSelector();
|
||||||
case label:
|
case label:
|
||||||
String expression = selectorJson.getString("expression");
|
String expression = selectorJson.get("expression").toString();
|
||||||
Set<String> labels = LabelSelector.parseExpression(expression);
|
Set<String> labels = LabelSelector.parseExpression(expression);
|
||||||
LabelSelector labelSelector = new LabelSelector();
|
LabelSelector labelSelector = new LabelSelector();
|
||||||
labelSelector.setExpression(expression);
|
labelSelector.setExpression(expression);
|
||||||
|
@ -15,84 +15,107 @@
|
|||||||
*/
|
*/
|
||||||
package com.alibaba.nacos.naming.controllers;
|
package com.alibaba.nacos.naming.controllers;
|
||||||
|
|
||||||
import com.alibaba.fastjson.JSONArray;
|
|
||||||
import com.alibaba.fastjson.JSONObject;
|
|
||||||
import com.alibaba.nacos.api.common.Constants;
|
import com.alibaba.nacos.api.common.Constants;
|
||||||
import com.alibaba.nacos.api.naming.CommonParams;
|
import com.alibaba.nacos.api.exception.NacosException;
|
||||||
import com.alibaba.nacos.naming.core.Cluster;
|
import com.alibaba.nacos.naming.core.Cluster;
|
||||||
|
import com.alibaba.nacos.naming.core.Instance;
|
||||||
import com.alibaba.nacos.naming.core.Service;
|
import com.alibaba.nacos.naming.core.Service;
|
||||||
import com.alibaba.nacos.naming.core.ServiceManager;
|
import com.alibaba.nacos.naming.core.ServiceManager;
|
||||||
import com.alibaba.nacos.naming.misc.UtilsAndCommons;
|
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||||
import org.junit.Assert;
|
|
||||||
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.runner.RunWith;
|
import org.junit.runner.RunWith;
|
||||||
import org.springframework.beans.factory.annotation.Autowired;
|
import org.mockito.Mock;
|
||||||
import org.springframework.boot.test.autoconfigure.web.servlet.AutoConfigureMockMvc;
|
import org.mockito.junit.MockitoJUnitRunner;
|
||||||
import org.springframework.boot.test.context.SpringBootTest;
|
|
||||||
import org.springframework.boot.test.mock.mockito.MockBean;
|
|
||||||
import org.springframework.test.context.junit4.SpringRunner;
|
|
||||||
import org.springframework.test.web.servlet.MockMvc;
|
|
||||||
|
|
||||||
import static org.mockito.Mockito.anyString;
|
import java.lang.reflect.Field;
|
||||||
|
import java.util.Collections;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
import static org.mockito.Mockito.when;
|
import static org.mockito.Mockito.when;
|
||||||
import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @author jifengnan 2019-04-29
|
* @author jifengnan 2019-04-29
|
||||||
*/
|
*/
|
||||||
@RunWith(SpringRunner.class)
|
@RunWith(MockitoJUnitRunner.class)
|
||||||
@SpringBootTest
|
|
||||||
@AutoConfigureMockMvc
|
|
||||||
public class CatalogControllerTest {
|
public class CatalogControllerTest {
|
||||||
|
|
||||||
@Autowired
|
@Mock
|
||||||
private MockMvc mockmvc;
|
|
||||||
|
|
||||||
@MockBean
|
|
||||||
private ServiceManager serviceManager;
|
private ServiceManager serviceManager;
|
||||||
|
|
||||||
@Test
|
private CatalogController catalogController;
|
||||||
public void testServiceDetail() throws Exception {
|
|
||||||
Service service = new Service(TEST_SERVICE_NAME);
|
private Service service;
|
||||||
|
|
||||||
|
private Cluster cluster;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws NoSuchFieldException, IllegalAccessException {
|
||||||
|
catalogController = new CatalogController();
|
||||||
|
Field field = catalogController.getClass().getDeclaredField("serviceManager");
|
||||||
|
field.setAccessible(true);
|
||||||
|
field.set(catalogController, serviceManager);
|
||||||
|
service = new Service(TEST_SERVICE_NAME);
|
||||||
service.setNamespaceId(Constants.DEFAULT_NAMESPACE_ID);
|
service.setNamespaceId(Constants.DEFAULT_NAMESPACE_ID);
|
||||||
service.setProtectThreshold(12.34f);
|
service.setProtectThreshold(12.34f);
|
||||||
service.setGroupName(TEST_GROUP_NAME);
|
service.setGroupName(TEST_GROUP_NAME);
|
||||||
Cluster cluster = new Cluster(TEST_CLUSTER_NAME, service);
|
cluster = new Cluster(TEST_CLUSTER_NAME, service);
|
||||||
cluster.setDefaultPort(1);
|
cluster.setDefaultPort(1);
|
||||||
|
|
||||||
service.addCluster(cluster);
|
service.addCluster(cluster);
|
||||||
when(serviceManager.getService(anyString(), anyString())).thenReturn(service);
|
when(serviceManager.getService(Constants.DEFAULT_NAMESPACE_ID, TEST_GROUP_NAME + Constants.SERVICE_INFO_SPLITER + TEST_SERVICE_NAME)).thenReturn(service);
|
||||||
String result1 = mockmvc.perform(get(UtilsAndCommons.NACOS_NAMING_CONTEXT + "/catalog/service")
|
|
||||||
.param(CommonParams.NAMESPACE_ID, Constants.DEFAULT_NAMESPACE_ID)
|
|
||||||
.param(CommonParams.SERVICE_NAME, TEST_SERVICE_NAME)
|
|
||||||
.param(CommonParams.GROUP_NAME, TEST_GROUP_NAME))
|
|
||||||
.andReturn().getResponse().getContentAsString();
|
|
||||||
JSONObject result = JSONObject.parseObject(result1);
|
|
||||||
JSONObject serviceResult = (JSONObject) result.get("service");
|
|
||||||
Assert.assertEquals(TEST_SERVICE_NAME, serviceResult.get("name"));
|
|
||||||
Assert.assertEquals(12.34, Float.parseFloat(serviceResult.get("protectThreshold").toString()), 0.01);
|
|
||||||
Assert.assertEquals(TEST_GROUP_NAME, serviceResult.get("groupName"));
|
|
||||||
|
|
||||||
JSONArray clusterResults = (JSONArray) result.get("clusters");
|
|
||||||
Assert.assertEquals(1, clusterResults.size());
|
|
||||||
JSONObject clusterResult = (JSONObject) clusterResults.get(0);
|
|
||||||
Assert.assertEquals(TEST_CLUSTER_NAME, clusterResult.get("name"));
|
|
||||||
Assert.assertEquals(1, Integer.parseInt(clusterResult.get("defaultPort").toString()));
|
|
||||||
Assert.assertTrue(Boolean.parseBoolean(clusterResult.get("useIPPort4Check").toString()));
|
|
||||||
Assert.assertEquals(TEST_SERVICE_NAME, clusterResult.get("serviceName"));
|
|
||||||
Assert.assertEquals(80, Integer.parseInt(clusterResult.get("defaultCheckPort").toString()));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testServiceDetailNotFound() throws Exception {
|
public void testServiceDetail() throws Exception {
|
||||||
String result = mockmvc.perform(get(UtilsAndCommons.NACOS_NAMING_CONTEXT + "/catalog/service")
|
ObjectNode result = catalogController.serviceDetail(Constants.DEFAULT_NAMESPACE_ID, TEST_GROUP_NAME + Constants.SERVICE_INFO_SPLITER + TEST_SERVICE_NAME);
|
||||||
.param(CommonParams.NAMESPACE_ID, Constants.DEFAULT_NAMESPACE_ID)
|
String actual = result.toString();
|
||||||
.param(CommonParams.SERVICE_NAME, TEST_SERVICE_NAME)).andReturn().getResponse().getContentAsString();
|
assertTrue(actual.contains("\"service\":{"));
|
||||||
|
assertTrue(actual.contains("\"groupName\":\"test-group-name\""));
|
||||||
|
assertTrue(actual.contains("\"metadata\":{}"));
|
||||||
|
assertTrue(actual.contains("\"name\":\"test-service\""));
|
||||||
|
assertTrue(actual.contains("\"selector\":{\"type\":\"none\"}"));
|
||||||
|
assertTrue(actual.contains("\"protectThreshold\":12.34"));
|
||||||
|
assertTrue(actual.contains("\"clusters\":[{"));
|
||||||
|
assertTrue(actual.contains("\"defaultCheckPort\":80"));
|
||||||
|
assertTrue(actual.contains("\"defaultPort\":1"));
|
||||||
|
assertTrue(actual.contains("\"healthChecker\":{\"type\":\"TCP\"}"));
|
||||||
|
assertTrue(actual.contains("\"metadata\":{}"));
|
||||||
|
assertTrue(actual.contains("\"name\":\"test-cluster\""));
|
||||||
|
assertTrue(actual.contains("\"serviceName\":\"test-service\""));
|
||||||
|
assertTrue(actual.contains("\"useIPPort4Check\":true"));
|
||||||
|
}
|
||||||
|
|
||||||
Assert.assertTrue(result.contains("test-service is not found!"));
|
@Test(expected = NacosException.class)
|
||||||
|
public void testServiceDetailNotFound() throws Exception {
|
||||||
|
catalogController.serviceDetail(Constants.DEFAULT_NAMESPACE_ID, TEST_SERVICE_NAME);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static final String TEST_CLUSTER_NAME = "test-cluster";
|
private static final String TEST_CLUSTER_NAME = "test-cluster";
|
||||||
private static final String TEST_SERVICE_NAME = "test-service";
|
private static final String TEST_SERVICE_NAME = "test-service";
|
||||||
private static final String TEST_GROUP_NAME = "test-group-name";
|
private static final String TEST_GROUP_NAME = "test-group-name";
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testInstanceList() throws NacosException {
|
||||||
|
Instance instance = new Instance("1.1.1.1", 1234, TEST_CLUSTER_NAME);
|
||||||
|
cluster.updateIPs(Collections.singletonList(instance), false);
|
||||||
|
ObjectNode result = catalogController.instanceList(Constants.DEFAULT_NAMESPACE_ID, TEST_GROUP_NAME + Constants.SERVICE_INFO_SPLITER + TEST_SERVICE_NAME,
|
||||||
|
TEST_CLUSTER_NAME, 1, 10);
|
||||||
|
String actual = result.toString();
|
||||||
|
assertTrue(actual.contains("\"count\":1"));
|
||||||
|
assertTrue(actual.contains("\"list\":["));
|
||||||
|
assertTrue(actual.contains("\"clusterName\":\"test-cluster\""));
|
||||||
|
assertTrue(actual.contains("\"ip\":\"1.1.1.1\""));
|
||||||
|
assertTrue(actual.contains("\"port\":1234"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testListDetail() {
|
||||||
|
// TODO
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRt4Service() {
|
||||||
|
// TODO
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -15,16 +15,16 @@
|
|||||||
*/
|
*/
|
||||||
package com.alibaba.nacos.naming.controllers;
|
package com.alibaba.nacos.naming.controllers;
|
||||||
|
|
||||||
import com.alibaba.fastjson.JSON;
|
|
||||||
import com.alibaba.fastjson.JSONArray;
|
|
||||||
import com.alibaba.fastjson.JSONObject;
|
|
||||||
import com.alibaba.nacos.api.common.Constants;
|
import com.alibaba.nacos.api.common.Constants;
|
||||||
|
import com.alibaba.nacos.common.utils.JacksonUtils;
|
||||||
import com.alibaba.nacos.naming.BaseTest;
|
import com.alibaba.nacos.naming.BaseTest;
|
||||||
import com.alibaba.nacos.naming.consistency.persistent.raft.RaftPeerSet;
|
import com.alibaba.nacos.naming.consistency.persistent.raft.RaftPeerSet;
|
||||||
import com.alibaba.nacos.naming.core.Cluster;
|
import com.alibaba.nacos.naming.core.Cluster;
|
||||||
import com.alibaba.nacos.naming.core.Instance;
|
import com.alibaba.nacos.naming.core.Instance;
|
||||||
import com.alibaba.nacos.naming.core.Service;
|
import com.alibaba.nacos.naming.core.Service;
|
||||||
import com.alibaba.nacos.naming.misc.UtilsAndCommons;
|
import com.alibaba.nacos.naming.misc.UtilsAndCommons;
|
||||||
|
import com.fasterxml.jackson.databind.JsonNode;
|
||||||
|
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
@ -135,18 +135,18 @@ public class InstanceControllerTest extends BaseTest {
|
|||||||
|
|
||||||
MockHttpServletResponse response = mockmvc.perform(builder).andReturn().getResponse();
|
MockHttpServletResponse response = mockmvc.perform(builder).andReturn().getResponse();
|
||||||
String actualValue = response.getContentAsString();
|
String actualValue = response.getContentAsString();
|
||||||
JSONObject result = JSON.parseObject(actualValue);
|
JsonNode result = JacksonUtils.toObj(actualValue);
|
||||||
|
|
||||||
Assert.assertEquals(TEST_SERVICE_NAME, result.getString("name"));
|
Assert.assertEquals(TEST_SERVICE_NAME, result.get("name").asText());
|
||||||
JSONArray hosts = result.getJSONArray("hosts");
|
JsonNode hosts = result.get("hosts");
|
||||||
Assert.assertNotNull(hosts);
|
Assert.assertNotNull(hosts);
|
||||||
Assert.assertEquals(hosts.size(), 1);
|
Assert.assertEquals(hosts.size(), 1);
|
||||||
|
|
||||||
JSONObject host = hosts.getJSONObject(0);
|
JsonNode host = hosts.get(0);
|
||||||
Assert.assertNotNull(host);
|
Assert.assertNotNull(host);
|
||||||
Assert.assertEquals("10.10.10.10", host.getString("ip"));
|
Assert.assertEquals("10.10.10.10", host.get("ip").asText());
|
||||||
Assert.assertEquals(8888, host.getIntValue("port"));
|
Assert.assertEquals(8888, host.get("port").asInt());
|
||||||
Assert.assertEquals(2.0, host.getDoubleValue("weight"), 0.001);
|
Assert.assertEquals(2.0, host.get("weight").asDouble(), 0.001);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@ -159,9 +159,9 @@ public class InstanceControllerTest extends BaseTest {
|
|||||||
|
|
||||||
MockHttpServletResponse response = mockmvc.perform(builder).andReturn().getResponse();
|
MockHttpServletResponse response = mockmvc.perform(builder).andReturn().getResponse();
|
||||||
String actualValue = response.getContentAsString();
|
String actualValue = response.getContentAsString();
|
||||||
JSONObject result = JSON.parseObject(actualValue);
|
JsonNode result = JacksonUtils.toObj(actualValue);
|
||||||
|
|
||||||
JSONArray hosts = result.getJSONArray("hosts");
|
JsonNode hosts = result.get("hosts");
|
||||||
Assert.assertEquals(hosts.size(), 0);
|
Assert.assertEquals(hosts.size(), 0);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user