You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ji...@apache.org on 2014/03/05 19:26:23 UTC
svn commit: r1574609 - in
/hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src:
main/java/org/apache/hadoop/ipc/ main/java/org/apache/hadoop/ipc/metrics/
test/java/org/apache/hadoop/ipc/
Author: jing9
Date: Wed Mar 5 18:26:22 2014
New Revision: 1574609
URL: http://svn.apache.org/r1574609
Log:
HDFS-5167. Merge change r1574608 from branch-2.
Added:
hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/metrics/RetryCacheMetrics.java
- copied unchanged from r1574608, hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/metrics/RetryCacheMetrics.java
hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRetryCacheMetrics.java
- copied unchanged from r1574608, hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRetryCacheMetrics.java
Modified:
hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RetryCache.java
Modified: hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RetryCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RetryCache.java?rev=1574609&r1=1574608&r2=1574609&view=diff
==============================================================================
--- hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RetryCache.java (original)
+++ hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RetryCache.java Wed Mar 5 18:26:22 2014
@@ -24,6 +24,7 @@ import java.util.UUID;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ipc.metrics.RetryCacheMetrics;
import org.apache.hadoop.util.LightWeightCache;
import org.apache.hadoop.util.LightWeightGSet;
import org.apache.hadoop.util.LightWeightGSet.LinkedElement;
@@ -43,6 +44,8 @@ import com.google.common.base.Preconditi
@InterfaceAudience.Private
public class RetryCache {
public static final Log LOG = LogFactory.getLog(RetryCache.class);
+ private final RetryCacheMetrics retryCacheMetrics;
+
/**
* CacheEntry is tracked using unique client ID and callId of the RPC request
*/
@@ -178,6 +181,7 @@ public class RetryCache {
private final LightWeightGSet<CacheEntry, CacheEntry> set;
private final long expirationTime;
+ private String cacheName;
/**
* Constructor
@@ -191,6 +195,8 @@ public class RetryCache {
this.set = new LightWeightCache<CacheEntry, CacheEntry>(capacity, capacity,
expirationTime, 0);
this.expirationTime = expirationTime;
+ this.cacheName = cacheName;
+ this.retryCacheMetrics = RetryCacheMetrics.create(this);
}
private static boolean skipRetryCache() {
@@ -199,12 +205,29 @@ public class RetryCache {
return !Server.isRpcInvocation() || Server.getCallId() < 0
|| Arrays.equals(Server.getClientId(), RpcConstants.DUMMY_CLIENT_ID);
}
-
+
+
+ private void incrCacheClearedCounter() {
+ retryCacheMetrics.incrCacheCleared();
+ }
+
@VisibleForTesting
public LightWeightGSet<CacheEntry, CacheEntry> getCacheSet() {
return set;
}
+ @VisibleForTesting
+ public RetryCacheMetrics getMetricsForTests() {
+ return retryCacheMetrics;
+ }
+
+ /**
+ * This method returns cache name for metrics.
+ */
+ public String getCacheName() {
+ return cacheName;
+ }
+
/**
* This method handles the following conditions:
* <ul>
@@ -234,7 +257,10 @@ public class RetryCache {
+ newEntry.callId + " to retryCache");
}
set.put(newEntry);
+ retryCacheMetrics.incrCacheUpdated();
return newEntry;
+ } else {
+ retryCacheMetrics.incrCacheHit();
}
}
// Entry already exists in cache. Wait for completion and return its state
@@ -269,6 +295,7 @@ public class RetryCache {
synchronized(this) {
set.put(newEntry);
}
+ retryCacheMetrics.incrCacheUpdated();
}
public void addCacheEntryWithPayload(byte[] clientId, int callId,
@@ -279,6 +306,7 @@ public class RetryCache {
synchronized(this) {
set.put(newEntry);
}
+ retryCacheMetrics.incrCacheUpdated();
}
private static CacheEntry newEntry(long expirationTime) {
@@ -330,6 +358,7 @@ public class RetryCache {
public static void clear(RetryCache cache) {
if (cache != null) {
cache.set.clear();
+ cache.incrCacheClearedCounter();
}
}
}