Skip to content

Commit

Permalink
GEODE-6588: Fixed mismatch of placeholders and arguments (apache#7375)
Browse files Browse the repository at this point in the history
  • Loading branch information
nabarunnag authored Jun 1, 2022
1 parent ea48e7f commit 992208e
Show file tree
Hide file tree
Showing 23 changed files with 54 additions and 54 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -263,15 +263,15 @@ public static void runTest1() throws Exception {
logger.debug("Naming Exception caught in lookup: " + e);
fail("failed in naming lookup: " + e);
} catch (Exception e) {
logger.debug("Exception caught during naming lookup: {}", e);
logger.debug("Exception caught during naming lookup:", e);
fail("failed in naming lookup: " + e);
}
try {
for (int count = 0; count < MAX_CONNECTIONS; count++) {
ds.getConnection();
}
} catch (Exception e) {
logger.debug("Exception caught in runTest1: {}", e);
logger.debug("Exception caught in runTest1:", e);
fail("Exception caught in runTest1: " + e);
}
} finally {
Expand Down Expand Up @@ -325,11 +325,11 @@ public String description() {
fail("expected a Login time-out exceeded");
} catch (SQLException sqle) {
if (sqle.getMessage().indexOf("Login time-out exceeded") == -1) {
logger.debug("Exception caught in runTest2: {}", sqle);
logger.debug("Exception caught in runTest2:", sqle);
fail("failed because of unhandled exception : " + sqle);
}
} catch (Exception e) {
logger.debug("Exception caught in runTest2: {}", e);
logger.debug("Exception caught in runTest2:", e);
fail("failed because of unhandled exception : " + e);
}
} finally {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -310,7 +310,7 @@ void sendPeriodicAck() {
success = true;
} catch (Exception ex) {
if (logger.isDebugEnabled()) {
logger.debug("Exception while sending an ack to the primary server: {}", ex);
logger.debug("Exception while sending an ack to the primary server:", ex);
}
} finally {
if (!success) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,7 @@ public static VersionedObjectList execute(ExecutablePool pool, Region region,
}
}
} catch (RuntimeException ex) {
logger.debug("single-hop removeAll encountered unexpected exception: {}", ex);
logger.debug("single-hop removeAll encountered unexpected exception:", ex);
throw ex;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ static int submitAllHA(List callableTasks, LocalRegion region, boolean isHA,
if (ee.getCause() instanceof InternalFunctionInvocationTargetException) {
if (isDebugEnabled) {
logger.debug(
"ExecuteRegionFunctionSingleHopOp#ExecutionException.InternalFunctionInvocationTargetException : Caused by :{}",
"ExecuteRegionFunctionSingleHopOp#ExecutionException.InternalFunctionInvocationTargetException: Caused by:",
ee.getCause());
}
try {
Expand Down Expand Up @@ -152,7 +152,7 @@ static int submitAllHA(List callableTasks, LocalRegion region, boolean isHA,
} else if (ee.getCause() instanceof FunctionException) {
if (isDebugEnabled) {
logger.debug(
"ExecuteRegionFunctionSingleHopOp#ExecutionException.FunctionException : Caused by :{}",
"ExecuteRegionFunctionSingleHopOp#ExecutionException.FunctionException: Caused by:",
ee.getCause());
}
FunctionException fe = (FunctionException) ee.getCause();
Expand All @@ -164,7 +164,7 @@ static int submitAllHA(List callableTasks, LocalRegion region, boolean isHA,
} else if (ee.getCause() instanceof ServerOperationException) {
if (isDebugEnabled) {
logger.debug(
"ExecuteRegionFunctionSingleHopOp#ExecutionException.ServerOperationException : Caused by :{}",
"ExecuteRegionFunctionSingleHopOp#ExecutionException.ServerOperationException: Caused by:",
ee.getCause());
}
ServerOperationException soe = (ServerOperationException) ee.getCause();
Expand Down Expand Up @@ -335,7 +335,7 @@ static Map<ServerLocation, Object> submitGetAll(
} catch (ExecutionException ee) {
if (ee.getCause() instanceof ServerOperationException) {
if (logger.isDebugEnabled()) {
logger.debug("GetAllOp#ExecutionException.ServerOperationException : Caused by :{}",
logger.debug("GetAllOp#ExecutionException.ServerOperationException: Caused by:",
ee.getCause());
}
throw (ServerOperationException) ee.getCause();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -909,7 +909,7 @@ void initializeHeldLocks(InternalDistributedMember owner, Set tokens)
// skipping because member is no longer in view
if (isDebugEnabled_DLS) {
logger.trace(LogMarker.DLS_VERBOSE,
"Initialization of held locks is skipping {} because owner {} is not in view: ",
"Initialization of held locks is skipping {} because owner {} is not in view: {}",
token, owner, members);
}
continue;
Expand Down Expand Up @@ -2234,7 +2234,7 @@ private boolean acquireReadLockPermission(DLockRequestMessage request) {
synchronized (suspendLock) {
checkDestroyed();
if (!dm.isCurrentMember(request.getSender())) {
logger.info(LogMarker.DLS_MARKER, "Ignoring lock request from non-member: %s", request);
logger.info(LogMarker.DLS_MARKER, "Ignoring lock request from non-member: {}", request);
return false;
}
Integer integer = (Integer) readLockCountMap.get(rThread);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,7 @@ private void initializeEventSeqNumQueue() {

if (logger.isDebugEnabled()) {
logger.debug(
"For bucket {} ,total keys recovered are : {} last key recovered is : {} and the seqNo is ",
"For bucket:{}, total keys recovered are:{} last key recovered is:{} and the seqNo is:{}",
getId(), eventSeqNumDeque.size(), lastKeyRecovered, getEventSeqNum());
}
}
Expand Down Expand Up @@ -188,11 +188,11 @@ private void destroyFailedBatchRemovalMessageKeys() {
// EntryNotFoundException to be thrown.
destroyKey(key, true);
if (isDebugEnabled) {
logger.debug("Destroyed {} from bucket: ", key, getId());
logger.debug("Destroyed {} from bucket: {}", key, getId());
}
} catch (ForceReattemptException fe) {
if (isDebugEnabled) {
logger.debug("Bucket :{} moved to other member", getId());
logger.debug("Bucket: {} moved to other member", getId());
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -326,10 +326,9 @@ protected void basicPerformTimeout(boolean isPending) throws CacheException {
}
if (hasExpired(getNow(), expTime)) {
if (logger.isTraceEnabled()) {
// NOTE: original finer message used this.toString() twice
logger.trace(
"{}.performTimeout().getExpirationTime() is {}; {}.expire({}). ttlExpiration: {}, idleExpiration: {}, ttlAttrs: {}, idleAttrs: {} action is: {}",
this, expTime, this, action, ttl, idle, getTTLAttributes(), getIdleAttributes());
"{}.performTimeout().getExpirationTime() is {}; ttlExpiration:{}, idleExpiration:{}, ttlAttrs:{}, idleAttrs:{} action is:{}",
this, expTime, ttl, idle, getTTLAttributes(), getIdleAttributes(), action);
}
expire(action, isPending);
return;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5148,12 +5148,12 @@ public static void validatePRID(InternalDistributedMember sender, int prId, Stri
PartitionedRegion pr = (PartitionedRegion) o;
if (pr.getPRId() == prId) {
if (!pr.getRegionIdentifier().equals(regionId)) {
logger.warn("{} is using PRID {} for {} but this process maps that PRID to {}",
new Object[] {sender.toString(), prId, pr.getRegionIdentifier()});
logger.warn("{} is using PRID {} for regionId {} but this process maps that PRID to {}",
sender, prId, regionId, pr.getRegionIdentifier());
}
} else if (pr.getRegionIdentifier().equals(regionId)) {
logger.warn("{} is using PRID {} for {} but this process is using PRID {}",
new Object[] {sender, prId, pr.getRegionIdentifier(), pr.getPRId()});
sender, prId, pr.getRegionIdentifier(), pr.getPRId());
}
}
}
Expand Down Expand Up @@ -5635,7 +5635,7 @@ public void cleanupFailedInitialization() {
}
if (savedFirstRuntimeException != null
&& savedFirstRuntimeException instanceof DistributedSystemDisconnectedException) {
logger.warn("cleanupFailedInitialization originally failed with {}",
logger.warn("cleanupFailedInitialization originally failed with:",
savedFirstRuntimeException);
throw (DistributedSystemDisconnectedException) savedFirstRuntimeException;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -597,7 +597,7 @@ public Object lockRedundancyLock(InternalDistributedMember moveSource, int bucke
ProxyBucketRegion buk = partitionedRegion.getRegionAdvisor().getProxyBucketArray()[bucketId];
if (!buk.checkBucketRedundancyBeforeGrab(moveSource, replaceOffineData)) {
if (logger.isDebugEnabled()) {
logger.debug("Redundancy already satisfied. current owners=",
logger.debug("Redundancy already satisfied. current owners:{}",
partitionedRegion.getRegionAdvisor().getBucketOwners(bucketId));
}
throw new RedundancyAlreadyMetException();
Expand Down Expand Up @@ -1257,7 +1257,7 @@ boolean canAccommodateMoreBytesSafely(int bytes) {
final long curBytes = bytesInUse.get();
if (isDebugEnabled) {
logger.debug(
"canAccomodateMoreBytes: bytes = {} allocatedMemory = {} newAllocatedSize = {} thresholdSize = ",
"canAccomodateMoreBytes: bytes = {} allocatedMemory = {} newAllocatedSize = {} thresholdSize = {}",
bytes, curBytes, (curBytes + bytes), maximumLocalBytes);
}
if ((curBytes + bytes) < maximumLocalBytes) {
Expand Down Expand Up @@ -2953,14 +2953,14 @@ public void executeOnDataStore(final Set localKeys, final Function function, fin
long start = stats.startFunctionExecution(function.hasResult());
try {
if (logger.isDebugEnabled()) {
logger.debug("Executing Function: {} on Remote Node with context: ", function.getId(),
logger.debug("Executing Function:{} on Remote Node with context:{}", function.getId(),
prContext);
}
function.execute(prContext);
stats.endFunctionExecution(start, function.hasResult());
} catch (FunctionException functionException) {
if (logger.isDebugEnabled()) {
logger.debug("FunctionException occurred on remote node while executing Function: {}",
logger.debug("FunctionException occurred on remote node while executing Function:{}",
function.getId(), functionException);
}
stats.endFunctionExecutionWithException(start, function.hasResult());
Expand Down Expand Up @@ -3022,7 +3022,7 @@ public void handleInterestEvent(InterestRegistrationEvent event) {
// If this is a registration event, add interest for this key
if (isRegister) {
if (logger.isDebugEnabled()) {
logger.debug("PartitionedRegionDataStore for {} adding interest for: ",
logger.debug("PartitionedRegionDataStore for {} adding interest for: {}",
partitionedRegion.getFullPath(), key);
}
if (references == null) {
Expand All @@ -3033,7 +3033,7 @@ public void handleInterestEvent(InterestRegistrationEvent event) {
} else {
// If this is an unregistration event, remove interest for this key
if (logger.isDebugEnabled()) {
logger.debug("PartitionedRegionDataStore for {} removing interest for: ",
logger.debug("PartitionedRegionDataStore for {} removing interest for: {}",
partitionedRegion.getFullPath(), key);
}
if (references != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -393,7 +393,7 @@ public static void send(InternalDistributedMember recipient, int processorId,
if (exception != null) {
m.setException(exception);
if (logger.isDebugEnabled()) {
logger.debug("Replying with exception: {}" + m, exception);
logger.debug("Replying with exception:" + m, exception);
}
}
m.setRecipient(recipient);
Expand Down Expand Up @@ -462,7 +462,7 @@ public static void send(InternalDistributedMember recipient, int processorId,
if (exception != null) {
m.setException(exception);
if (logger.isDebugEnabled()) {
logger.debug("Replying with exception: {}" + m, exception);
logger.debug("Replying with exception:" + m, exception);
}
}
m.setRecipient(recipient);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -98,7 +98,7 @@ public synchronized void lastResult(Object oneResult) {
}

if (logger.isDebugEnabled()) {
logger.debug("ServerToClientFunctionResultSender sending last result1 {} " + oneResult);
logger.debug("ServerToClientFunctionResultSender sending last result1 {}", oneResult);
}
try {
authorizeResult(oneResult);
Expand Down Expand Up @@ -145,7 +145,7 @@ public synchronized void lastResult(Object oneResult, DistributedMember memberID
return;
}
if (logger.isDebugEnabled()) {
logger.debug("ServerToClientFunctionResultSender sending last result2 {} " + oneResult);
logger.debug("ServerToClientFunctionResultSender sending last result2 {}", oneResult);
}
try {
authorizeResult(oneResult);
Expand Down Expand Up @@ -192,7 +192,7 @@ public synchronized void sendResult(Object oneResult) {
return;
}
if (logger.isDebugEnabled()) {
logger.debug("ServerToClientFunctionResultSender sending result1 {} " + oneResult);
logger.debug("ServerToClientFunctionResultSender sending result1 {}", oneResult);
}
try {
authorizeResult(oneResult);
Expand Down Expand Up @@ -234,7 +234,7 @@ public synchronized void sendResult(Object oneResult, DistributedMember memberID
return;
}
if (logger.isDebugEnabled()) {
logger.debug("ServerToClientFunctionResultSender sending result2 {} " + oneResult);
logger.debug("ServerToClientFunctionResultSender sending result2 {}", oneResult);
}
try {
authorizeResult(oneResult);
Expand Down Expand Up @@ -310,7 +310,7 @@ public synchronized void setException(Throwable exception) {
return;
}
if (logger.isDebugEnabled()) {
logger.debug("ServerToClientFunctionResultSender setting exception {} ", exception);
logger.debug("ServerToClientFunctionResultSender setting exception:", exception);
}
synchronized (msg) {
if (!sc.getTransientFlag(Command.RESPONDED)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,7 @@ public synchronized void lastResult(Object oneResult, DistributedMember memberID
}
try {
if (logger.isDebugEnabled()) {
logger.debug("ServerToClientFunctionResultSender sending last result2 {} " + oneResult);
logger.debug("ServerToClientFunctionResultSender sending last result2 {}", oneResult);
}
authorizeResult(oneResult);
if (!fn.hasResult()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -108,8 +108,9 @@ private void sendRecoveryMsgs(final DistributionManager dm, final DLockBatch[] b
// this shouldn't happen unless we're shutting down or someone has set a size constraint
// on the waiting-pool using a system property
if (!dm.getCancelCriterion().isCancelInProgress()) {
logger.warn("Unable to schedule background cleanup of transactions for departed member {}."
+ " Performing in-line cleanup of the transactions.");
logger.warn(
"Unable to schedule background cleanup of transactions for departed member {}. Performing in-line cleanup of the transactions.",
owner);
recoverTx.run();
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,7 @@ protected boolean operateOnPartitionedRegion(ClusterDistributionManager dm, Part
ContainsKeyValueReplyMessage.send(getSender(), getProcessorId(), getReplySender(dm),
replyVal);
} else {
logger.fatal("Partitioned Region <> is not configured to store data",
logger.fatal("Partitioned Region {} is not configured to store data",
r.getFullPath());
ForceReattemptException fre = new ForceReattemptException(
String.format("Partitioned Region %s on %s is not configured to store data",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ public synchronized boolean sendReplyForOneResult(DistributionManager dm, Partit
sendResultsInOrder);

if (logger.isDebugEnabled()) {
logger.debug("Sending reply message count: {} to co-ordinating node");
logger.debug("Sending reply message count: {} to co-ordinating node", msgNum);
}

replyMsgNum++;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1304,7 +1304,7 @@ public void accept() {
closeSocket(socket);
if (isRunning()) {
if (logger.isDebugEnabled()) {
logger.debug("Aborted due to interrupt: {}", e);
logger.debug("Aborted due to interrupt:", e);
}
}
} catch (IOException e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ public void cmdExecute(final @NotNull Message clientMessage,
String regionName = regionNamePart.getCachedString();

if (regionName == null) {
logger.warn("The input region name for the %s request is null", "size");
logger.warn("The input region name for the size request is null");
writeErrorResponse(clientMessage, MessageType.SIZE_ERROR,
"The input region name for the size request is null",
serverConnection);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -161,7 +161,7 @@ public void process(final DistributionManager dm, final ReplyProcessor21 process
final long startTime = getTimestamp();
if (logger.isTraceEnabled(LogMarker.DM_VERBOSE)) {
logger.trace(LogMarker.DM_VERBOSE,
"{}: process invoking reply processor with processorId: {}", processorId);
"{}: process invoking reply processor with processorId: {}", this, processorId);
}

if (processor == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -131,7 +131,7 @@ public ClientConnectionFactoryWrapper getManagedDataSource(Map configMap,
cf = mcf.createConnectionFactory((ConnectionManager) cm);
} catch (Exception ex) {
logger.error(
"DataSourceFactory::getManagedDataSource: Exception in creating managed connection factory. Exception string, %s",
"DataSourceFactory::getManagedDataSource: Exception in creating managed connection factory. Exception string:{}",
ex.toString());
throw new DataSourceCreateException(
String.format(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ public void afterUpdate(EntryEvent<String, Object> event) {

} catch (Exception e) {
if (logger.isDebugEnabled()) {
logger.debug("Aggregation Failed failed for {} with exception {}", e);
logger.debug("Aggregation Failed failed for " + objectName + " with exception", e);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -769,8 +769,8 @@ void memberJoined(InternalDistributedMember id) {
try {
listener.memberJoined(event);
} catch (Exception e) {
logger.error("Could not invoke listener event memberJoined for listener[{}] due to ",
listener.getClass(), e.getMessage(), e);
logger.error("Could not invoke listener event memberJoined for listener["
+ listener.getClass() + "] due to " + e.getMessage(), e);
}
}
}
Expand All @@ -782,17 +782,17 @@ void memberDeparted(InternalDistributedMember id, boolean crashed) {
try {
listener.memberCrashed(event);
} catch (Exception e) {
logger.error("Could not invoke listener event memberCrashed for listener[{}] due to ",
listener.getClass(), e.getMessage(), e);
logger.error("Could not invoke listener event memberCrashed for listener["
+ listener.getClass() + "] due to " + e.getMessage(), e);
}
}
} else {
for (MembershipListener listener : membershipListeners) {
try {
listener.memberLeft(event);
} catch (Exception e) {
logger.error("Could not invoke listener event memberLeft for listener[{}] due to ",
listener.getClass(), e.getMessage(), e);
logger.error("Could not invoke listener event memberLeft for listener["
+ listener.getClass() + "] due to " + e.getMessage(), e);
}
}
}
Expand Down
Loading

0 comments on commit 992208e

Please sign in to comment.