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