incubator-hama-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From edwardy...@apache.org
Subject svn commit: r1102562 - in /incubator/hama/trunk: CHANGES.txt src/java/org/apache/hama/bsp/BSPPeer.java src/java/org/apache/hama/bsp/ByteMessage.java src/test/org/apache/hama/bsp/TestBSPPeer.java src/test/org/apache/hama/bsp/TestMessages.java
Date Fri, 13 May 2011 04:54:07 GMT
Author: edwardyoon
Date: Fri May 13 04:54:07 2011
New Revision: 1102562

URL: http://svn.apache.org/viewvc?rev=1102562&view=rev
Log:
Removing 'System.arraycopy(...)' call to reduce memory/garbage collection usage

Added:
    incubator/hama/trunk/src/test/org/apache/hama/bsp/TestMessages.java
Modified:
    incubator/hama/trunk/CHANGES.txt
    incubator/hama/trunk/src/java/org/apache/hama/bsp/BSPPeer.java
    incubator/hama/trunk/src/java/org/apache/hama/bsp/ByteMessage.java
    incubator/hama/trunk/src/test/org/apache/hama/bsp/TestBSPPeer.java

Modified: incubator/hama/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/CHANGES.txt?rev=1102562&r1=1102561&r2=1102562&view=diff
==============================================================================
--- incubator/hama/trunk/CHANGES.txt (original)
+++ incubator/hama/trunk/CHANGES.txt Fri May 13 04:54:07 2011
@@ -11,6 +11,7 @@ Release 0.3 - Unreleased
 
   IMPROVEMENTS
 
+    HAMA-386: Removing 'System.arraycopy(...)' call from ByteMessage (edwardyoon)
     HAMA-379: Setting default values to job.setNumBspTask() (Thomas Jungblut via edwardyoon)
     HAMA-376: Refactor BSPMaster and GroomServer (ChiaHung Lin via edwardyoon)
     HAMA-382: Refactor HAMA POM (Tommaso Teofili)

Modified: incubator/hama/trunk/src/java/org/apache/hama/bsp/BSPPeer.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/bsp/BSPPeer.java?rev=1102562&r1=1102561&r2=1102562&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/bsp/BSPPeer.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/bsp/BSPPeer.java Fri May 13 04:54:07 2011
@@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC.Server;
 import org.apache.hama.Constants;
+import org.apache.hama.util.Bytes;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;

Modified: incubator/hama/trunk/src/java/org/apache/hama/bsp/ByteMessage.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/bsp/ByteMessage.java?rev=1102562&r1=1102561&r2=1102562&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/bsp/ByteMessage.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/bsp/ByteMessage.java Fri May 13 04:54:07
2011
@@ -32,22 +32,18 @@ public class ByteMessage extends BSPMess
 
   public ByteMessage(byte[] tag, byte[] data) {
     super();
-    this.tag = new byte[tag.length];
-    this.data = new byte[data.length];
-    System.arraycopy(tag, 0, this.tag, 0, tag.length);
-    System.arraycopy(data, 0, this.data, 0, data.length);
+    this.tag = tag;
+    this.data = data;
   }
 
   @Override
   public byte[] getTag() {
-    byte[] result = this.tag;
-    return result;
+    return this.tag;
   }
 
   @Override
   public byte[] getData() {
-    byte[] result = this.data;
-    return result;
+    return this.data;
   }
 
   @Override

Modified: incubator/hama/trunk/src/test/org/apache/hama/bsp/TestBSPPeer.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/test/org/apache/hama/bsp/TestBSPPeer.java?rev=1102562&r1=1102561&r2=1102562&view=diff
==============================================================================
--- incubator/hama/trunk/src/test/org/apache/hama/bsp/TestBSPPeer.java (original)
+++ incubator/hama/trunk/src/test/org/apache/hama/bsp/TestBSPPeer.java Fri May 13 04:54:07
2011
@@ -47,7 +47,7 @@ import org.apache.zookeeper.data.Stat;
 public class TestBSPPeer extends HamaCluster implements Watcher {
   private Log LOG = LogFactory.getLog(TestBSPPeer.class);
 
-  private static final int NUM_PEER = 10;
+  private static final int NUM_PEER = 3;
   private static final int ROUND = 3;
   private static final int PAYLOAD = 1024; // 1kb in default
   List<BSPPeerThread> list = new ArrayList<BSPPeerThread>(NUM_PEER);
@@ -105,16 +105,16 @@ public class TestBSPPeer extends HamaClu
     @Override
     public void runTest() throws AssertionFailedError {
       int randomTime;
-      byte[] dummyData = new byte[PAYLOAD];
-      BSPMessage msg = null;
 
       for (int i = 0; i < ROUND; i++) {
         randomTime = r.nextInt(MAXIMUM_DURATION) + 5;
 
-        for (int j = 0; j < 10; j++) {
+        for (int j = 0; j < 3; j++) {
+          byte[] dummyData = new byte[PAYLOAD];
           r.nextBytes(dummyData);
-          msg = new ByteMessage(Bytes.tail(dummyData, 128), dummyData);
           String peerName = "localhost:" + (30000 + j);
+          ByteMessage msg = new ByteMessage(Bytes.tail(dummyData, 128),
+              dummyData);
           try {
             peer.send(peerName, msg);
           } catch (IOException e) {
@@ -149,8 +149,8 @@ public class TestBSPPeer extends HamaClu
       LOG.info("[" + peer.getPeerName() + "] verifying " + numMessages
           + " messages at " + round + " round");
 
-      if (lastTwoDigitsOfPort < 10) {
-        assertEquals(10, numMessages);
+      if (lastTwoDigitsOfPort < 3) {
+        assertEquals(3, numMessages);
       } else {
         assertEquals(0, numMessages);
       }

Added: incubator/hama/trunk/src/test/org/apache/hama/bsp/TestMessages.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/test/org/apache/hama/bsp/TestMessages.java?rev=1102562&view=auto
==============================================================================
--- incubator/hama/trunk/src/test/org/apache/hama/bsp/TestMessages.java (added)
+++ incubator/hama/trunk/src/test/org/apache/hama/bsp/TestMessages.java Fri May 13 04:54:07
2011
@@ -0,0 +1,21 @@
+package org.apache.hama.bsp;
+
+import junit.framework.TestCase;
+
+import org.apache.hama.util.Bytes;
+
+public class TestMessages extends TestCase {
+
+  public void testByteMessage() {
+    int dataSize = (int) (Runtime.getRuntime().maxMemory() * 0.60);
+    ByteMessage msg = new ByteMessage(Bytes.toBytes("tag"), new byte[dataSize]);
+    assertEquals(msg.getData().length, dataSize);
+    msg = null;
+    
+    byte[] dummyData = new byte[1024];
+    ByteMessage msg2 = new ByteMessage(Bytes.tail(dummyData, 128), dummyData);
+    assertEquals(
+        Bytes.compareTo(msg2.getTag(), 0, 128, msg2.getData(),
+            msg2.getData().length - 128, 128), 0);
+  }
+}



Mime
View raw message