You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ds...@apache.org on 2015/08/13 00:32:44 UTC

[2/2] incubator-geode git commit: no longer call System.currentTimeMillis every time off-heap memory changes

no longer call System.currentTimeMillis every time off-heap memory changes


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/2b4e7f64
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/2b4e7f64
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/2b4e7f64

Branch: refs/heads/feature/GEODE-212
Commit: 2b4e7f64d8913293b1691973ad90b9577d975b5e
Parents: 3b3acc7
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Wed Aug 12 15:05:23 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Wed Aug 12 15:05:23 2015 -0700

----------------------------------------------------------------------
 .../internal/cache/control/MemoryEvent.java     |  7 --
 .../cache/control/OffHeapMemoryMonitor.java     | 88 +++++++++++++-------
 2 files changed, 56 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2b4e7f64/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/MemoryEvent.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/MemoryEvent.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/MemoryEvent.java
index 2e4468f..c6dfd10 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/MemoryEvent.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/MemoryEvent.java
@@ -25,7 +25,6 @@ public class MemoryEvent implements ResourceEvent {
   private final long bytesUsed;
   private final boolean isLocal;
   private final MemoryThresholds thresholds;
-  private final long eventTime;
  
   public MemoryEvent(final ResourceType type, final MemoryState previousState, final MemoryState state,
       final DistributedMember member, final long bytesUsed, final boolean isLocal, final MemoryThresholds thresholds) {
@@ -36,7 +35,6 @@ public class MemoryEvent implements ResourceEvent {
     this.bytesUsed = bytesUsed;
     this.isLocal = isLocal;
     this.thresholds = thresholds;
-    this.eventTime = System.currentTimeMillis();
   }
 
   @Override
@@ -66,10 +64,6 @@ public class MemoryEvent implements ResourceEvent {
     return this.isLocal;
   }
   
-  public long getEventTime() {
-    return this.eventTime;
-  }
-
   public MemoryThresholds getThresholds() {
     return this.thresholds;
   }
@@ -84,7 +78,6 @@ public class MemoryEvent implements ResourceEvent {
         .append(",state:" + this.state)
         .append(",bytesUsed:" + this.bytesUsed)
         .append(",isLocal:" + this.isLocal)
-        .append(",eventTime:" + this.eventTime)
         .append(",thresholds:" + this.thresholds + "]")
         .toString();
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2b4e7f64/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/OffHeapMemoryMonitor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/OffHeapMemoryMonitor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/OffHeapMemoryMonitor.java
index bd29f88..0678c01 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/OffHeapMemoryMonitor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/OffHeapMemoryMonitor.java
@@ -239,26 +239,24 @@ public class OffHeapMemoryMonitor implements ResourceMonitor, MemoryUsageListene
    */
   public void updateStateAndSendEvent(long bytesUsed) {
     synchronized (this) {
-      MemoryState oldState = this.mostRecentEvent.getState();
-      MemoryState newState = this.thresholds.computeNextState(oldState, bytesUsed);
+      final MemoryEvent mre = this.mostRecentEvent;
+      final MemoryState oldState = mre.getState();
+      final MemoryThresholds thresholds = this.thresholds;
+      MemoryState newState = thresholds.computeNextState(oldState, bytesUsed);
       if (oldState != newState) {
         this.currentState = newState;
         
-        MemoryEvent event = new MemoryEvent(ResourceType.OFFHEAP_MEMORY, oldState, newState, this.cache.getMyId(), bytesUsed, true,
-            this.thresholds);
+        MemoryEvent event = new MemoryEvent(ResourceType.OFFHEAP_MEMORY, oldState, newState, this.cache.getMyId(), bytesUsed, true, thresholds);
         this.upcomingEvent.set(event);
 
         processLocalEvent(event);
         updateStatsFromEvent(event);
         
-      // The state didn't change.  However, if the state isn't normal and we've
-      // been in that state for a while, send another event with the updated
-      // memory usage.
       } else if (!oldState.isNormal()
-          && bytesUsed != this.mostRecentEvent.getBytesUsed()
-          && (System.currentTimeMillis() - this.mostRecentEvent.getEventTime()) > 1000) {
-        MemoryEvent event = new MemoryEvent(ResourceType.OFFHEAP_MEMORY, oldState, newState, this.cache.getMyId(), bytesUsed, true,
-            this.thresholds);
+          && bytesUsed != mre.getBytesUsed()
+          && this.deliverNextAbnormalEvent) {
+        this.deliverNextAbnormalEvent = false;
+        MemoryEvent event = new MemoryEvent(ResourceType.OFFHEAP_MEMORY, oldState, newState, this.cache.getMyId(), bytesUsed, true, thresholds);
         this.upcomingEvent.set(event);
         processLocalEvent(event);
       }
@@ -273,21 +271,33 @@ public class OffHeapMemoryMonitor implements ResourceMonitor, MemoryUsageListene
    *          Number of bytes of off-heap memory currently used.
    * @return true if a new event might need to be sent
    */
-   private boolean mightSendEvent(long bytesUsed) {
-     MemoryEvent mre = this.mostRecentEvent;
-     MemoryState oldState = mre.getState();
-     MemoryState newState = this.thresholds.computeNextState(oldState, bytesUsed);
-     if (oldState != newState) {
-       return true;
-     // The state didn't change.  However, if the state isn't normal and we've
-     // been in that state for a while, send another event with the updated
-     // memory usage.
-     } else if (!oldState.isNormal()
-         && bytesUsed != mre.getBytesUsed()
-         && (System.currentTimeMillis() - mre.getEventTime()) > 1000) {
-       return true;
-     }
-     return false;
+  private boolean mightSendEvent(long bytesUsed) {
+    final MemoryEvent mre = this.mostRecentEvent;
+    final MemoryState oldState = mre.getState();
+    final MemoryThresholds thresholds = this.thresholds;
+    MemoryState newState = thresholds.computeNextState(oldState, bytesUsed);
+    if (oldState != newState) {
+      return true;
+    } else if (!oldState.isNormal()
+        && bytesUsed != mre.getBytesUsed()
+        && this.deliverNextAbnormalEvent) {
+      return true;
+    }
+    return false;
+  }
+  
+  private volatile boolean deliverNextAbnormalEvent = false;
+   
+  /**
+   * Used by the OffHeapMemoryUsageListener to tell us that
+   * the next abnormal event should be delivered even if the
+   * state does not change as long as the memory usage changed.
+   * For some reason, unknown to me, if we stay in an abnormal
+   * state for more than a second then we want to send another
+   * event to update the memory usage.
+   */
+  void deliverNextAbnormalEvent() {
+    this.deliverNextAbnormalEvent = true;
   }
 
   /**
@@ -417,7 +427,7 @@ public class OffHeapMemoryMonitor implements ResourceMonitor, MemoryUsageListene
   
   class OffHeapMemoryUsageListener implements Runnable {
     volatile boolean stopRequested = false;
-    volatile long offHeapMemoryUsed; // In bytes
+    long offHeapMemoryUsed; // In bytes
     
     OffHeapMemoryUsageListener(final long offHeapMemoryUsed) {
       this.offHeapMemoryUsed = offHeapMemoryUsed;
@@ -426,15 +436,29 @@ public class OffHeapMemoryMonitor implements ResourceMonitor, MemoryUsageListene
     @Override
     public void run() {
       getLogWriter().fine("OffHeapMemoryUsageListener is starting " + this);
-
+      long lastOffHeapMemoryUsed;
+      synchronized (this) {
+        lastOffHeapMemoryUsed = this.offHeapMemoryUsed;
+      }
       while (!this.stopRequested) {
-        final long saveOffHeapMemoryUsed = this.offHeapMemoryUsed;
-        updateStateAndSendEvent(saveOffHeapMemoryUsed);
+        updateStateAndSendEvent(lastOffHeapMemoryUsed);
 
         synchronized (this) {
-          if (saveOffHeapMemoryUsed == this.offHeapMemoryUsed && !this.stopRequested) {
+          if (lastOffHeapMemoryUsed == this.offHeapMemoryUsed && !this.stopRequested) {
             try {
-              this.wait();
+              do {
+                this.wait(1000);
+                if (this.offHeapMemoryUsed == lastOffHeapMemoryUsed) {
+                  // The wait timed out. So tell the OffHeapMemoryMonitor
+                  // that we need an event if the state is not normal.
+                  deliverNextAbnormalEvent();
+                } else {
+                  // we have been notified so exit the inner while loop
+                  // and call updateStateAndSendEvent.
+                  lastOffHeapMemoryUsed = this.offHeapMemoryUsed;
+                  break;
+                }
+              } while (true);
             } catch (InterruptedException iex) {
               getLogWriter().warning("OffHeapMemoryUsageListener was interrupted " + this);
               this.stopRequested = true;