Skip to content

Commit

Permalink
IGNITE-14823 Request, response abbrevation (#11159)
Browse files Browse the repository at this point in the history
  • Loading branch information
nizhikov authored Dec 29, 2023
1 parent 1c32eb2 commit ae7eb66
Show file tree
Hide file tree
Showing 14 changed files with 69 additions and 69 deletions.
4 changes: 2 additions & 2 deletions modules/checkstyle/src/main/resources/abbrevations.csv
Original file line number Diff line number Diff line change
Expand Up @@ -57,9 +57,9 @@ receive,rcv
reference,ref
#regularexpression,regex
repository,repo
#request,req
request,req
resource,rsrc
#response,res
response,res
#returnValue,retVal
#sender,snd
#service,srvc
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,22 +58,22 @@ public static Map<String, Object> executeProbeRestRequest() throws IOException {

boolean isHTTP_OK = conn.getResponseCode() == HttpURLConnection.HTTP_OK;

Map<String, Object> restResponse = null;
Map<String, Object> restRes = null;

try (InputStreamReader streamReader = new InputStreamReader(isHTTP_OK ? conn.getInputStream() : conn.getErrorStream())) {

ObjectMapper objMapper = new ObjectMapper();
restResponse = objMapper.readValue(streamReader,
restRes = objMapper.readValue(streamReader,
new TypeReference<Map<String, Object>>() {
});

log.info("probe command response is: " + restResponse);
log.info("probe command response is: " + restRes);

}
catch (Exception e) {
log.error("error executing probe rest command", e);
}
return restResponse;
return restRes;

}

Expand Down Expand Up @@ -146,21 +146,21 @@ public void testRestProbeCommandGridNotStarted() throws Exception {
}
}).start();

Map<String, Object> probeRestCmdResponse;
Map<String, Object> probeRestCmdRes;

log.info("awaiting plugin handler latch");
triggerPluginStartLatch.await();
log.info("starting rest command url call");
try {
probeRestCmdResponse = executeProbeRestRequest();
probeRestCmdRes = executeProbeRestRequest();
log.info("finished rest command url call");
}
finally {
triggerRestCmdLatch.countDown(); //make sure the grid shuts down
}

assertTrue(probeRestCmdResponse.get("error").equals("grid has not started"));
assertEquals(GridRestResponse.SERVICE_UNAVAILABLE, probeRestCmdResponse.get("successStatus"));
assertTrue(probeRestCmdRes.get("error").equals("grid has not started"));
assertEquals(GridRestResponse.SERVICE_UNAVAILABLE, probeRestCmdRes.get("successStatus"));
}

/**
Expand All @@ -172,12 +172,12 @@ public void testRestProbeCommandGridNotStarted() throws Exception {
public void testRestProbeCommandGridStarted() throws Exception {
startGrid("regular");

Map<String, Object> probeRestCmdResponse;
Map<String, Object> probeRestCmdRes;

probeRestCmdResponse = executeProbeRestRequest();
probeRestCmdRes = executeProbeRestRequest();

assertTrue(probeRestCmdResponse.get("response").equals("grid has started"));
assertEquals(0, probeRestCmdResponse.get("successStatus"));
assertTrue(probeRestCmdRes.get("response").equals("grid has started"));
assertEquals(0, probeRestCmdRes.get("successStatus"));
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -400,7 +400,7 @@ private ClientRequestFuture send(ClientOperation op, Consumer<PayloadOutputChann
*/
private <T> T receive(ClientRequestFuture pendingReq, Function<PayloadInputChannel, T> payloadReader)
throws ClientException {
long requestId = pendingReq.requestId;
long reqId = pendingReq.requestId;
ClientOperation op = pendingReq.operation;
long startTimeNanos = pendingReq.startTimeNanos;

Expand All @@ -411,7 +411,7 @@ private <T> T receive(ClientRequestFuture pendingReq, Function<PayloadInputChann
if (payload != null && payloadReader != null)
res = payloadReader.apply(new PayloadInputChannel(this, payload));

eventListener.onRequestSuccess(connDesc, requestId, op.code(), op.name(), System.nanoTime() - startTimeNanos);
eventListener.onRequestSuccess(connDesc, reqId, op.code(), op.name(), System.nanoTime() - startTimeNanos);

return res;
}
Expand All @@ -420,7 +420,7 @@ private <T> T receive(ClientRequestFuture pendingReq, Function<PayloadInputChann

RuntimeException err = convertException(e);

eventListener.onRequestFail(connDesc, requestId, op.code(), op.name(), System.nanoTime() - startTimeNanos, err);
eventListener.onRequestFail(connDesc, reqId, op.code(), op.name(), System.nanoTime() - startTimeNanos, err);

throw err;
}
Expand All @@ -435,7 +435,7 @@ private <T> T receive(ClientRequestFuture pendingReq, Function<PayloadInputChann
*/
private <T> CompletableFuture<T> receiveAsync(ClientRequestFuture pendingReq, Function<PayloadInputChannel, T> payloadReader) {
CompletableFuture<T> fut = new CompletableFuture<>();
long requestId = pendingReq.requestId;
long reqId = pendingReq.requestId;
ClientOperation op = pendingReq.operation;
long startTimeNanos = pendingReq.startTimeNanos;

Expand All @@ -447,7 +447,7 @@ private <T> CompletableFuture<T> receiveAsync(ClientRequestFuture pendingReq, Fu
if (payload != null && payloadReader != null)
res = payloadReader.apply(new PayloadInputChannel(this, payload));

eventListener.onRequestSuccess(connDesc, requestId, op.code(), op.name(), System.nanoTime() - startTimeNanos);
eventListener.onRequestSuccess(connDesc, reqId, op.code(), op.name(), System.nanoTime() - startTimeNanos);

fut.complete(res);
}
Expand All @@ -456,7 +456,7 @@ private <T> CompletableFuture<T> receiveAsync(ClientRequestFuture pendingReq, Fu

RuntimeException err = convertException(t);

eventListener.onRequestFail(connDesc, requestId, op.code(), op.name(), System.nanoTime() - startTimeNanos, err);
eventListener.onRequestFail(connDesc, reqId, op.code(), op.name(), System.nanoTime() - startTimeNanos, err);

fut.completeExceptionally(err);
}
Expand Down Expand Up @@ -697,7 +697,7 @@ private static void validateConfiguration(ClientChannelConfiguration cfg) {
/** Client handshake. */
private void handshake(ProtocolVersion ver, String user, String pwd, Map<String, String> userAttrs)
throws ClientConnectionException, ClientAuthenticationException, ClientProtocolError {
long requestId = -1L;
long reqId = -1L;
long startTime = System.nanoTime();

eventListener.onHandshakeStart(new ConnectionDescription(sock.localAddress(), sock.remoteAddress(),
Expand All @@ -712,9 +712,9 @@ private void handshake(ProtocolVersion ver, String user, String pwd, Map<String,
if (closed())
throw new ClientConnectionException("Channel is closed");

fut = new ClientRequestFuture(requestId, ClientOperation.HANDSHAKE);
fut = new ClientRequestFuture(reqId, ClientOperation.HANDSHAKE);

pendingReqs.put(requestId, fut);
pendingReqs.put(reqId, fut);
}
finally {
pendingReqsLock.readLock().unlock();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -613,7 +613,7 @@ private GridByteArrayList sendClassRequest(String name, String path) throws Clas
nodeLdrMapCp = singleNode ? nodeLdrMap : new HashMap<>(nodeLdrMap);
}

List<IgniteException> clsRequestExceptions = new ArrayList<>();
List<IgniteException> clsReqExceptions = new ArrayList<>();

for (UUID nodeId : nodeListCp) {
if (nodeId.equals(ctx.discovery().localNode().id()))
Expand Down Expand Up @@ -644,7 +644,7 @@ private GridByteArrayList sendClassRequest(String name, String path) throws Clas

LT.warn(log, msg);

clsRequestExceptions.add(new IgniteException(msg));
clsReqExceptions.add(new IgniteException(msg));

synchronized (mux) {
if (missedRsrcs != null)
Expand Down Expand Up @@ -683,20 +683,20 @@ else if (log.isDebugEnabled())
else if (log.isDebugEnabled())
log.debug(msg);

clsRequestExceptions.add(new IgniteException(msg, e));
clsReqExceptions.add(new IgniteException(msg, e));
}
}
catch (TimeoutException e) {
clsRequestExceptions.add(new IgniteException("Failed to send class-loading request to node (is node alive?) " +
clsReqExceptions.add(new IgniteException("Failed to send class-loading request to node (is node alive?) " +
"[node=" + node.id() + ", clsName=" + name + ", clsPath=" + path + ", clsLdrId=" + ldrId +
", clsLoadersHierarchy=" + clsLdrHierarchy + ']', e));
}
}

if (!clsRequestExceptions.isEmpty()) {
IgniteException ex = clsRequestExceptions.remove(0);
if (!clsReqExceptions.isEmpty()) {
IgniteException ex = clsReqExceptions.remove(0);

for (Exception e : clsRequestExceptions)
for (Exception e : clsReqExceptions)
ex.addSuppressed(e);

LT.warn(log, ex.getMessage(), ex);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -697,7 +697,7 @@ void addGroupKey(int grpId, GroupKeyEncrypted key) {
"Unable to get the master key digest.", e));
}

MasterKeyChangeRequest request = new MasterKeyChangeRequest(UUID.randomUUID(), encryptKeyName(masterKeyName),
MasterKeyChangeRequest req = new MasterKeyChangeRequest(UUID.randomUUID(), encryptKeyName(masterKeyName),
digest);

synchronized (opsMux) {
Expand All @@ -717,9 +717,9 @@ void addGroupKey(int grpId, GroupKeyEncrypted key) {
"The previous change was not completed."));
}

masterKeyChangeFut = new KeyChangeFuture(request.requestId());
masterKeyChangeFut = new KeyChangeFuture(req.requestId());

prepareMKChangeProc.start(request.requestId(), request);
prepareMKChangeProc.start(req.requestId(), req);

return new IgniteFutureImpl<>(masterKeyChangeFut);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2345,7 +2345,7 @@ private void dumpLongRunningTransaction(IgniteInternalTx tx) {

ClusterGroup nearNode = ignite.cluster().forNodeId(nearNodeId);

String txRequestInfo = String.format(
String txReqInfo = String.format(
"[xidVer=%s, nodeId=%s]",
tx.xidVersion().toString(),
nearNodeId.toString()
Expand All @@ -2369,13 +2369,13 @@ private void dumpLongRunningTransaction(IgniteInternalTx tx) {
U.error(
diagnosticLog,
"Could not get thread dump from transaction owner because near node " +
"is out of topology now. " + txRequestInfo
"is out of topology now. " + txReqInfo
);
}
catch (Exception e) {
U.error(
diagnosticLog,
"Could not get thread dump from transaction owner near node " + txRequestInfo,
"Could not get thread dump from transaction owner near node " + txReqInfo,
e
);
}
Expand All @@ -2385,7 +2385,7 @@ private void dumpLongRunningTransaction(IgniteInternalTx tx) {
diagnosticLog,
String.format(
"Dumping the near node thread that started transaction %s\n%s",
txRequestInfo,
txReqInfo,
traceDump
)
);
Expand All @@ -2394,14 +2394,14 @@ private void dumpLongRunningTransaction(IgniteInternalTx tx) {
});
}
catch (Exception e) {
U.error(diagnosticLog, "Could not send dump request to transaction owner near node " + txRequestInfo, e);
U.error(diagnosticLog, "Could not send dump request to transaction owner near node " + txReqInfo, e);
}
}
else {
U.warn(
diagnosticLog,
"Could not send dump request to transaction owner near node: node does not support this feature. " +
txRequestInfo
txReqInfo
);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1376,11 +1376,11 @@ private IgniteInternalFuture<Void> completeHandlersAsyncIfNeeded(SnapshotOperati

Map<String, List<SnapshotHandlerResult<?>>> clusterHndResults = new HashMap<>();

for (SnapshotOperationResponse response : res) {
if (response == null || response.handlerResults() == null)
for (SnapshotOperationResponse snpRes : res) {
if (snpRes == null || snpRes.handlerResults() == null)
continue;

for (Map.Entry<String, SnapshotHandlerResult<Object>> entry : response.handlerResults().entrySet())
for (Map.Entry<String, SnapshotHandlerResult<Object>> entry : snpRes.handlerResults().entrySet())
clusterHndResults.computeIfAbsent(entry.getKey(), v -> new ArrayList<>()).add(entry.getValue());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1538,11 +1538,11 @@ private void processChangeGlobalStateResponse(final UUID nodeId, final GridChang
", nodeId=" + nodeId + "]");
}

UUID requestId = msg.getRequestId();
UUID reqId = msg.getRequestId();

final GridChangeGlobalStateFuture fut = stateChangeFut.get();

if (fut != null && requestId.equals(fut.requestId)) {
if (fut != null && reqId.equals(fut.requestId)) {
if (fut.initFut.isDone())
fut.onResponse(nodeId, msg);
else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -598,10 +598,10 @@ public void cancelQuery(long queryId, @Nullable UUID nodeId, boolean async) {

cancellationRuns.put(reqId, fut);

final GridQueryKillRequest request = new GridQueryKillRequest(reqId, queryId, async);
final GridQueryKillRequest req = new GridQueryKillRequest(reqId, queryId, async);

if (node.isLocal() && !async)
locNodeMsgHnd.apply(node, request);
locNodeMsgHnd.apply(node, req);
else {
try {
if (node.isLocal()) {
Expand All @@ -611,7 +611,7 @@ public void cancelQuery(long queryId, @Nullable UUID nodeId, boolean async) {
return;

try {
locNodeMsgHnd.apply(node, request);
locNodeMsgHnd.apply(node, req);
}
finally {
busyLock.leaveBusy();
Expand All @@ -620,7 +620,7 @@ public void cancelQuery(long queryId, @Nullable UUID nodeId, boolean async) {
}, GridIoPolicy.MANAGEMENT_POOL);
}
else {
ctx.io().sendGeneric(node, GridTopic.TOPIC_QUERY, GridTopic.TOPIC_QUERY.ordinal(), request,
ctx.io().sendGeneric(node, GridTopic.TOPIC_QUERY, GridTopic.TOPIC_QUERY.ordinal(), req,
GridIoPolicy.MANAGEMENT_POOL);
}
}
Expand Down Expand Up @@ -755,20 +755,20 @@ else if (!runningQryInfo.cancelable())
* @param err Error message
*/
private void sendKillResponse(GridQueryKillRequest request, ClusterNode node, @Nullable String err) {
GridQueryKillResponse response = new GridQueryKillResponse(request.requestId(), err);
GridQueryKillResponse res = new GridQueryKillResponse(request.requestId(), err);

if (node.isLocal()) {
locNodeMsgHnd.apply(node, response);
locNodeMsgHnd.apply(node, res);

return;
}

try {
ctx.io().sendGeneric(node, GridTopic.TOPIC_QUERY, GridTopic.TOPIC_QUERY.ordinal(), response,
ctx.io().sendGeneric(node, GridTopic.TOPIC_QUERY, GridTopic.TOPIC_QUERY.ordinal(), res,
GridIoPolicy.MANAGEMENT_POOL);
}
catch (IgniteCheckedException e) {
U.warn(log, "Failed to send message [node=" + node + ", msg=" + response +
U.warn(log, "Failed to send message [node=" + node + ", msg=" + res +
", errMsg=" + e.getMessage() + "]");

U.warn(log, "Response on query cancellation wasn't send back: [qryId=" + request.nodeQryId() + "]");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -337,13 +337,13 @@ private boolean isInterruptible(GridClientMessage msg) {
if (!(msg instanceof GridClientTaskRequest))
return false;

GridClientTaskRequest taskRequest = (GridClientTaskRequest)msg;
GridClientTaskRequest taskReq = (GridClientTaskRequest)msg;

try {
return U.hasAnnotation(U.forName(taskRequest.taskName(), null), InterruptibleVisorTask.class);
return U.hasAnnotation(U.forName(taskReq.taskName(), null), InterruptibleVisorTask.class);
}
catch (ClassNotFoundException e) {
log.warning("Task closure can't be found: [task=" + taskRequest.taskName() + ']', e);
log.warning("Task closure can't be found: [task=" + taskReq.taskName() + ']', e);

return false;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1815,10 +1815,10 @@ private boolean writeToBuffer(GridSelectorNioSessionImpl ses, ByteBuffer buf, Se
* @param requests SessionWriteRequests.
*/
private void onRequestsWritten(GridSelectorNioSessionImpl ses, List<SessionWriteRequest> requests) {
for (SessionWriteRequest request : requests) {
request.onMessageWritten();
for (SessionWriteRequest req : requests) {
req.onMessageWritten();

onMessageWritten(ses, (Message)request.message());
onMessageWritten(ses, (Message)req.message());
}
}

Expand Down
Loading

0 comments on commit ae7eb66

Please sign in to comment.