Repository: incubator-reef
Updated Branches:
  refs/heads/master d09250e2e -> 3e8b862b4


[REEF-651] Remove unnecessary logging information from reef-io.

Currently, many REEF-IO methods are logging unnecessary information when
entering and exiting. That seriously affects performance and can easily
cause OutOfMemory exceptions, e.g. when logging a big ML model. This
change cleans that up. Whenever entering or exiting is called, we won't
log any potential big object. If users decide to log that information in
their own application, so be it. But this shouldn't be done in the
communications library.

JIRA:
  [REEF-651](https://issues.apache.org/jira/browse/REEF-651)

Pull Request:
  Closes #420


Project: http://git-wip-us.apache.org/repos/asf/incubator-reef/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-reef/commit/3e8b862b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-reef/tree/3e8b862b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-reef/diff/3e8b862b

Branch: refs/heads/master
Commit: 3e8b862b4570ec4e5555dc05f2ed4e7c73dc985e
Parents: d09250e
Author: Ignacio Cano <[email protected]>
Authored: Wed Aug 26 15:26:55 2015 -0700
Committer: Markus Weimer <[email protected]>
Committed: Wed Aug 26 15:36:20 2015 -0700

----------------------------------------------------------------------
 .../group/impl/operators/BroadcastReceiver.java |  5 +--
 .../group/impl/operators/BroadcastSender.java   |  7 ++--
 .../group/impl/operators/GatherReceiver.java    | 13 +++---
 .../group/impl/operators/ReduceReceiver.java    |  5 +--
 .../group/impl/operators/ReduceSender.java      |  6 +--
 .../group/impl/operators/ScatterSender.java     | 16 ++++----
 .../io/network/group/impl/operators/Sender.java |  5 +--
 .../group/impl/task/OperatorTopologyImpl.java   | 29 ++++++-------
 .../impl/task/OperatorTopologyStructImpl.java   | 43 ++++++++------------
 9 files changed, 54 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3e8b862b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastReceiver.java
----------------------------------------------------------------------
diff --git 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastReceiver.java
 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastReceiver.java
index 5a3e426..671b06f 100644
--- 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastReceiver.java
+++ 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastReceiver.java
@@ -41,7 +41,7 @@ import org.apache.reef.tang.annotations.Parameter;
 import org.apache.reef.wake.EventHandler;
 
 import javax.inject.Inject;
-import java.util.Arrays;
+
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.logging.Logger;
 
@@ -144,7 +144,6 @@ public class BroadcastReceiver<T> implements 
Broadcast.Receiver<T>, EventHandler
         LOG.finest("Using " + dataCodec.getClass().getSimpleName() + " as 
codec");
         retVal = dataCodec.decode(data);
         LOG.finest("Decoded msg successfully");
-        LOG.fine(this + " Received: " + retVal);
         LOG.finest(this + " Sending to children.");
       }
 
@@ -152,7 +151,7 @@ public class BroadcastReceiver<T> implements 
Broadcast.Receiver<T>, EventHandler
     } catch (final ParentDeadException e) {
       throw new RuntimeException("ParentDeadException", e);
     }
-    LOG.exiting("BroadcastReceiver", "receive", Arrays.toString(new 
Object[]{retVal, this}));
+    LOG.exiting("BroadcastReceiver", "receive", this);
     return retVal;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3e8b862b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastSender.java
----------------------------------------------------------------------
diff --git 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastSender.java
 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastSender.java
index 7f2d0d0..76178de 100644
--- 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastSender.java
+++ 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastSender.java
@@ -41,7 +41,7 @@ import org.apache.reef.tang.annotations.Parameter;
 import org.apache.reef.wake.EventHandler;
 
 import javax.inject.Inject;
-import java.util.Arrays;
+
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.logging.Logger;
 
@@ -120,7 +120,7 @@ public class BroadcastSender<T> implements 
Broadcast.Sender<T>, EventHandler<Gro
 
   @Override
   public void send(final T element) throws NetworkException, 
InterruptedException {
-    LOG.entering("BroadcastSender", "send", new Object[]{this, element});
+    LOG.entering("BroadcastSender", "send", this);
     LOG.fine("I am " + this);
 
     if (init.compareAndSet(false, true)) {
@@ -130,12 +130,11 @@ public class BroadcastSender<T> implements 
Broadcast.Sender<T>, EventHandler<Gro
     }
 
     try {
-      LOG.fine(this + " Broadcasting " + element);
       topology.sendToChildren(dataCodec.encode(element), 
ReefNetworkGroupCommProtos.GroupCommMessage.Type.Broadcast);
     } catch (final ParentDeadException e) {
       throw new RuntimeException("ParentDeadException", e);
     }
-    LOG.exiting("BroadcastSender", "send", Arrays.toString(new Object[]{this, 
element}));
+    LOG.exiting("BroadcastSender", "send", this);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3e8b862b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/GatherReceiver.java
----------------------------------------------------------------------
diff --git 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/GatherReceiver.java
 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/GatherReceiver.java
index 4b217f6..834fb79 100644
--- 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/GatherReceiver.java
+++ 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/GatherReceiver.java
@@ -46,6 +46,7 @@ import java.io.DataInputStream;
 import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.Level;
 import java.util.logging.Logger;
 
 public class GatherReceiver<T> implements Gather.Receiver<T>, 
EventHandler<GroupCommunicationMessage> {
@@ -122,20 +123,20 @@ public class GatherReceiver<T> implements 
Gather.Receiver<T>, EventHandler<Group
     LOG.entering("GatherReceiver", "receive");
     final Map<String, T> mapOfTaskIdToData = receiveMapOfTaskIdToData();
 
-    LOG.fine(this + " Sorting data according to lexicographical order of task 
identifiers.");
+    LOG.log(Level.FINE, "{0} Sorting data according to lexicographical order 
of task identifiers.", this);
     final TreeMap<String, T> sortedMapOfTaskIdToData = new 
TreeMap<>(mapOfTaskIdToData);
     final List<T> retList = new LinkedList<>(sortedMapOfTaskIdToData.values());
 
-    LOG.exiting("GatherReceiver", "receive", retList);
+    LOG.exiting("GatherReceiver", "receive");
     return retList;
   }
 
   @Override
   public List<T> receive(final List<? extends Identifier> order) throws 
NetworkException, InterruptedException {
-    LOG.entering("GatherReceiver", "receive", order);
+    LOG.entering("GatherReceiver", "receive");
     final Map<String, T> mapOfTaskIdToData = receiveMapOfTaskIdToData();
 
-    LOG.fine(this + " Sorting data according to specified order of task 
identifiers.");
+    LOG.log(Level.FINE, "{0} Sorting data according to specified order of task 
identifiers.", this);
     final List<T> retList = new LinkedList<>();
     for (final Identifier key : order) {
       final String keyString = key.toString();
@@ -147,7 +148,7 @@ public class GatherReceiver<T> implements 
Gather.Receiver<T>, EventHandler<Group
       }
     }
 
-    LOG.exiting("GatherReceiver", "receive", retList);
+    LOG.exiting("GatherReceiver", "receive");
     return retList;
   }
 
@@ -186,7 +187,7 @@ public class GatherReceiver<T> implements 
Gather.Receiver<T>, EventHandler<Group
       throw new RuntimeException("IOException", e);
     }
 
-    LOG.exiting("GatherReceiver", "receiveMapOfTaskIdToData", 
Arrays.toString(new Object[]{mapOfTaskIdToData, this}));
+    LOG.exiting("GatherReceiver", "receiveMapOfTaskIdToData");
     return mapOfTaskIdToData;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3e8b862b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceReceiver.java
----------------------------------------------------------------------
diff --git 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceReceiver.java
 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceReceiver.java
index 7fd2fc6..e4ce44e 100644
--- 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceReceiver.java
+++ 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceReceiver.java
@@ -39,7 +39,7 @@ import org.apache.reef.wake.EventHandler;
 import org.apache.reef.wake.Identifier;
 
 import javax.inject.Inject;
-import java.util.Arrays;
+
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.logging.Logger;
@@ -137,8 +137,7 @@ public class ReduceReceiver<T> implements 
Reduce.Receiver<T>, EventHandler<Group
     } catch (final ParentDeadException e) {
       throw new RuntimeException("ParentDeadException", e);
     }
-    LOG.fine(this + " Received Reduced value: " + (redVal != null ? redVal : 
"NULL"));
-    LOG.exiting("ReduceReceiver", "reduce", Arrays.toString(new 
Object[]{redVal}));
+    LOG.exiting("ReduceReceiver", "reduce", this);
     return redVal;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3e8b862b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceSender.java
----------------------------------------------------------------------
diff --git 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceSender.java
 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceSender.java
index e62deb8..7eaaf43 100644
--- 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceSender.java
+++ 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceSender.java
@@ -40,7 +40,6 @@ import org.apache.reef.wake.EventHandler;
 
 import javax.inject.Inject;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.logging.Level;
@@ -129,7 +128,7 @@ public class ReduceSender<T> implements Reduce.Sender<T>, 
EventHandler<GroupComm
 
   @Override
   public void send(final T myData) throws NetworkException, 
InterruptedException {
-    LOG.entering("ReduceSender", "send", new Object[]{this, myData});
+    LOG.entering("ReduceSender", "send", this);
     LOG.fine("I am " + this);
 
     if (init.compareAndSet(false, true)) {
@@ -146,12 +145,11 @@ public class ReduceSender<T> implements Reduce.Sender<T>, 
EventHandler<GroupComm
         vals.add(reducedValueOfChildren);
       }
       final T reducedValue = reduceFunction.apply(vals);
-      LOG.fine(this + " Sending local " + reducedValue + " to parent");
       topology.sendToParent(dataCodec.encode(reducedValue), 
ReefNetworkGroupCommProtos.GroupCommMessage.Type.Reduce);
     } catch (final ParentDeadException e) {
       throw new RuntimeException("ParentDeadException", e);
     }
-    LOG.exiting("ReduceSender", "send", Arrays.toString(new Object[]{this, 
myData}));
+    LOG.exiting("ReduceSender", "send", this);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3e8b862b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ScatterSender.java
----------------------------------------------------------------------
diff --git 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ScatterSender.java
 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ScatterSender.java
index bef15f4..617f8ac 100644
--- 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ScatterSender.java
+++ 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ScatterSender.java
@@ -132,20 +132,20 @@ public final class ScatterSender<T> implements 
Scatter.Sender<T>, EventHandler<G
 
   @Override
   public void send(final List<T> elements) throws NetworkException, 
InterruptedException {
-    LOG.entering("ScatterSender", "send", elements);
+    LOG.entering("ScatterSender", "send");
 
     initializeGroup();
     send(elements,
         ScatterHelper.getUniformCounts(elements.size(), 
commGroupClient.getActiveSlaveTasks().size()),
         commGroupClient.getActiveSlaveTasks());
 
-    LOG.exiting("ScatterSender", "send", elements);
+    LOG.exiting("ScatterSender", "send");
   }
 
   @Override
   public void send(final List<T> elements, final Integer... counts)
       throws NetworkException, InterruptedException {
-    LOG.entering("ScatterSender", "send", new Object[]{elements, counts});
+    LOG.entering("ScatterSender", "send");
 
     initializeGroup();
     if (counts.length != commGroupClient.getActiveSlaveTasks().size()) {
@@ -157,26 +157,26 @@ public final class ScatterSender<T> implements 
Scatter.Sender<T>, EventHandler<G
         Arrays.asList(counts),
         commGroupClient.getActiveSlaveTasks());
 
-    LOG.exiting("ScatterSender", "send", Arrays.toString(new 
Object[]{elements, counts}));
+    LOG.exiting("ScatterSender", "send");
   }
 
   @Override
   public void send(final List<T> elements, final List<? extends Identifier> 
order)
       throws NetworkException, InterruptedException {
-    LOG.entering("ScatterSender", "send", new Object[]{elements, order});
+    LOG.entering("ScatterSender", "send");
 
     initializeGroup();
     send(elements,
         ScatterHelper.getUniformCounts(elements.size(), order.size()),
         order);
 
-    LOG.exiting("ScatterSender", "send", Arrays.toString(new 
Object[]{elements, order}));
+    LOG.exiting("ScatterSender", "send");
   }
 
   @Override
   public void send(final List<T> elements, final List<Integer> counts, final 
List<? extends Identifier> order)
       throws NetworkException, InterruptedException {
-    LOG.entering("ScatterSender", "send", new Object[]{elements, counts, 
order});
+    LOG.entering("ScatterSender", "send");
 
     if (counts.size() != order.size()) {
       throw new RuntimeException("Parameter 'counts' has size " + counts.size()
@@ -198,6 +198,6 @@ public final class ScatterSender<T> implements 
Scatter.Sender<T>, EventHandler<G
       throw new RuntimeException("ParentDeadException during 
OperatorTopology.sendToChildren()", e);
     }
 
-    LOG.exiting("ScatterSender", "send", Arrays.toString(new Object[]{this, 
elements, counts, order}));
+    LOG.exiting("ScatterSender", "send");
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3e8b862b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/Sender.java
----------------------------------------------------------------------
diff --git 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/Sender.java
 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/Sender.java
index e4513db..02d35b8 100644
--- 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/Sender.java
+++ 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/Sender.java
@@ -27,7 +27,6 @@ import 
org.apache.reef.io.network.util.StringIdentifierFactory;
 import org.apache.reef.wake.Identifier;
 import org.apache.reef.wake.IdentifierFactory;
 
-import java.util.Arrays;
 import java.util.logging.Logger;
 
 public class Sender extends AbstractGroupCommOperator {
@@ -49,11 +48,11 @@ public class Sender extends AbstractGroupCommOperator {
   }
 
   public void send(final GroupCommunicationMessage msg, final String dest) 
throws NetworkException {
-    LOG.entering("Sender", "send", new Object[]{msg, dest});
+    LOG.entering("Sender", "send", msg);
     final Identifier destId = idFac.getNewInstance(dest);
     final Connection<GroupCommunicationMessage> link = 
netService.newConnection(destId);
     link.open();
     link.write(msg);
-    LOG.exiting("Sender", "send", Arrays.toString(new Object[]{msg, dest}));
+    LOG.exiting("Sender", "send", msg);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3e8b862b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyImpl.java
----------------------------------------------------------------------
diff --git 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyImpl.java
 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyImpl.java
index d64fa5c..b38e1e6 100644
--- 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyImpl.java
+++ 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyImpl.java
@@ -199,45 +199,42 @@ public class OperatorTopologyImpl implements 
OperatorTopology {
   @Override
   public void sendToParent(final byte[] data, final 
ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType)
       throws ParentDeadException {
-    LOG.entering("OperatorTopologyImpl", "sendToParent", new 
Object[]{getQualifiedName(), data, msgType});
+    LOG.entering("OperatorTopologyImpl", "sendToParent", new Object[] 
{getQualifiedName(), msgType});
     refreshEffectiveTopology();
     assert (effectiveTopology != null);
     effectiveTopology.sendToParent(data, msgType);
-    LOG.exiting("OperatorTopologyImpl", "sendToParent",
-        Arrays.toString(new Object[]{getQualifiedName(), data, msgType}));
+    LOG.exiting("OperatorTopologyImpl", "sendToParent", getQualifiedName());
   }
 
   @Override
   public void sendToChildren(final byte[] data, final 
ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType)
       throws ParentDeadException {
-    LOG.entering("OperatorTopologyImpl", "sendToChildren", new 
Object[]{getQualifiedName(), data, msgType});
+    LOG.entering("OperatorTopologyImpl", "sendToChildren", new 
Object[]{getQualifiedName(), msgType});
     refreshEffectiveTopology();
     assert (effectiveTopology != null);
     effectiveTopology.sendToChildren(data, msgType);
-    LOG.exiting("OperatorTopologyImpl", "sendToChildren",
-        Arrays.toString(new Object[]{getQualifiedName(), data, msgType}));
+    LOG.exiting("OperatorTopologyImpl", "sendToChildren", getQualifiedName());
   }
 
   @Override
   public void sendToChildren(final Map<String, byte[]> dataMap,
                              final 
ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType)
       throws ParentDeadException {
-    LOG.entering("OperatorTopologyImpl", "sendToChildren", new 
Object[]{getQualifiedName(), dataMap, msgType});
+    LOG.entering("OperatorTopologyImpl", "sendToChildren", new 
Object[]{getQualifiedName(), msgType});
     refreshEffectiveTopology();
     assert (effectiveTopology != null);
     effectiveTopology.sendToChildren(dataMap, msgType);
-    LOG.exiting("OperatorTopologyImpl", "sendToChildren",
-        Arrays.toString(new Object[]{getQualifiedName(), dataMap, msgType}));
+    LOG.exiting("OperatorTopologyImpl", "sendToChildren", getQualifiedName());
   }
 
   @Override
   public byte[] recvFromParent(final 
ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType)
       throws ParentDeadException {
-    LOG.entering("OperatorTopologyImpl", "recvFromParent", getQualifiedName());
+    LOG.entering("OperatorTopologyImpl", "recvFromParent", new Object[] 
{getQualifiedName(), msgType});
     refreshEffectiveTopology();
     assert (effectiveTopology != null);
     final byte[] retVal = effectiveTopology.recvFromParent(msgType);
-    LOG.exiting("OperatorTopologyImpl", "recvFromParent", Arrays.toString(new 
Object[]{getQualifiedName(), retVal}));
+    LOG.exiting("OperatorTopologyImpl", "recvFromParent", getQualifiedName());
     return retVal;
   }
 
@@ -248,7 +245,7 @@ public class OperatorTopologyImpl implements 
OperatorTopology {
     refreshEffectiveTopology();
     assert (effectiveTopology != null);
     final T retVal = effectiveTopology.recvFromChildren(redFunc, dataCodec);
-    LOG.exiting("OperatorTopologyImpl", "recvFromChildren", 
Arrays.toString(new Object[]{getQualifiedName(), retVal}));
+    LOG.exiting("OperatorTopologyImpl", "recvFromChildren", 
getQualifiedName());
     return retVal;
   }
 
@@ -258,7 +255,7 @@ public class OperatorTopologyImpl implements 
OperatorTopology {
     refreshEffectiveTopology();
     assert (effectiveTopology != null);
     final byte[] retVal = effectiveTopology.recvFromChildren();
-    LOG.exiting("OperatorTopologyImpl", "recvFromChildren", 
Arrays.toString(new Object[]{getQualifiedName(), retVal}));
+    LOG.exiting("OperatorTopologyImpl", "recvFromChildren", 
getQualifiedName());
     return retVal;
   }
 
@@ -411,8 +408,7 @@ public class OperatorTopologyImpl implements 
OperatorTopology {
    */
   private void copyDeletionDeltas(final Set<GroupCommunicationMessage> 
deletionDeltasForUpdate)
       throws ParentDeadException {
-    LOG.entering("OperatorTopologyImpl", "copyDeletionDeltas", new 
Object[]{getQualifiedName(),
-        deletionDeltasForUpdate});
+    LOG.entering("OperatorTopologyImpl", "copyDeletionDeltas", 
getQualifiedName());
     this.deletionDeltas.drainTo(deletionDeltasForUpdate);
     for (final GroupCommunicationMessage msg : deletionDeltasForUpdate) {
       final ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType = 
msg.getType();
@@ -421,8 +417,7 @@ public class OperatorTopologyImpl implements 
OperatorTopology {
             "Parent dead. Current behavior is for the child to die too.");
       }
     }
-    LOG.exiting("OperatorTopologyImpl", "copyDeletionDeltas", 
Arrays.toString(new Object[]{getQualifiedName(),
-        deletionDeltasForUpdate}));
+    LOG.exiting("OperatorTopologyImpl", "copyDeletionDeltas", 
getQualifiedName());
   }
 
   private String getQualifiedName() {

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/3e8b862b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyStructImpl.java
----------------------------------------------------------------------
diff --git 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyStructImpl.java
 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyStructImpl.java
index 83afde4..18afeed 100644
--- 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyStructImpl.java
+++ 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyStructImpl.java
@@ -177,7 +177,7 @@ public class OperatorTopologyStructImpl implements 
OperatorTopologyStruct {
   private void sendToNode(final byte[] data,
                           final 
ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType,
                           final NodeStruct node) {
-    LOG.entering("OperatorTopologyStructImpl", "sendToNode", new 
Object[]{getQualifiedName(), data, msgType, node});
+    LOG.entering("OperatorTopologyStructImpl", "sendToNode", new 
Object[]{getQualifiedName(), msgType, node});
     final String nodeId = node.getId();
     try {
 
@@ -190,8 +190,7 @@ public class OperatorTopologyStructImpl implements 
OperatorTopologyStruct {
           LOG.finest(getQualifiedName() + "Got readiness to accept " + msgType 
+ " msg from " + nodeId
               + ". Will send actual msg now");
         } else {
-          LOG.exiting("OperatorTopologyStructImpl", "sendToNode", 
Arrays.toString(new Object[]{getQualifiedName(),
-              data, msgType, node}));
+          LOG.exiting("OperatorTopologyStructImpl", "sendToNode", 
getQualifiedName());
           return;
         }
       }
@@ -206,8 +205,7 @@ public class OperatorTopologyStructImpl implements 
OperatorTopologyStruct {
           LOG.finest(getQualifiedName() + "Got " + msgType + " msg received 
ACK from " + nodeId
               + ". Will move to next msg if it exists");
         } else {
-          LOG.exiting("OperatorTopologyStructImpl", "sendToNode", 
Arrays.toString(new Object[]{getQualifiedName(),
-              data, msgType, node}));
+          LOG.exiting("OperatorTopologyStructImpl", "sendToNode", 
getQualifiedName());
           return;
         }
       }
@@ -216,8 +214,7 @@ public class OperatorTopologyStructImpl implements 
OperatorTopologyStruct {
           "NetworkException while sending " + msgType + " data from " + selfId 
+ " to " + nodeId,
           e);
     }
-    LOG.exiting("OperatorTopologyStructImpl", "sendToNode", 
Arrays.toString(new Object[]{getQualifiedName(), data,
-        msgType, node}));
+    LOG.exiting("OperatorTopologyStructImpl", "sendToNode", 
getQualifiedName());
   }
 
   private byte[] receiveFromNode(final NodeStruct node, final boolean remove) {
@@ -233,8 +230,7 @@ public class OperatorTopologyStructImpl implements 
OperatorTopologyStruct {
         LOG.fine(msg);
       }
     }
-    LOG.exiting("OperatorTopologyStructImpl", "receiveFromNode",
-        Arrays.toString(new Object[]{retVal, getQualifiedName(), node, 
remove}));
+    LOG.exiting("OperatorTopologyStructImpl", "receiveFromNode", 
getQualifiedName());
     return retVal;
   }
 
@@ -270,8 +266,7 @@ public class OperatorTopologyStructImpl implements 
OperatorTopologyStruct {
       }
     }
 
-    LOG.exiting("OperatorTopologyStructImpl", "recvFromNodeCheckBigMsg",
-        Arrays.toString(new Object[]{retVal, node, msgType}));
+    LOG.exiting("OperatorTopologyStructImpl", "recvFromNodeCheckBigMsg");
     return retVal;
   }
 
@@ -296,30 +291,28 @@ public class OperatorTopologyStructImpl implements 
OperatorTopologyStruct {
 
   @Override
   public void sendToParent(final byte[] data, final 
ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType) {
-    LOG.entering("OperatorTopologyStructImpl", "sendToParent", new 
Object[]{getQualifiedName(), data, msgType});
+    LOG.entering("OperatorTopologyStructImpl", "sendToParent", new 
Object[]{getQualifiedName(), msgType});
     if (parent != null) {
       sendToNode(data, msgType, parent);
     } else {
       LOG.fine(getQualifiedName() + "Perhaps parent has died or has not been 
configured");
     }
-    LOG.exiting("OperatorTopologyStructImpl", "sendToParent", 
Arrays.toString(new Object[]{getQualifiedName(), data,
-        msgType}));
+    LOG.exiting("OperatorTopologyStructImpl", "sendToParent", 
getQualifiedName());
   }
 
   @Override
   public void sendToChildren(final byte[] data, final 
ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType) {
-    LOG.entering("OperatorTopologyStructImpl", "sendToChildren", new 
Object[]{getQualifiedName(), data, msgType});
+    LOG.entering("OperatorTopologyStructImpl", "sendToChildren", new 
Object[]{getQualifiedName(), msgType});
     for (final NodeStruct child : children) {
       sendToNode(data, msgType, child);
     }
-    LOG.exiting("OperatorTopologyStructImpl", "sendToChildren", 
Arrays.toString(new Object[]{getQualifiedName(),
-        data, msgType}));
+    LOG.exiting("OperatorTopologyStructImpl", "sendToChildren", 
getQualifiedName());
   }
 
   @Override
   public void sendToChildren(final Map<String, byte[]> dataMap,
                              final 
ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType) {
-    LOG.entering("OperatorTopologyStructImpl", "sendToChildren", new 
Object[]{getQualifiedName(), dataMap, msgType});
+    LOG.entering("OperatorTopologyStructImpl", "sendToChildren", new 
Object[]{getQualifiedName(), msgType});
     for (final NodeStruct child : children) {
       if (dataMap.containsKey(child.getId())) {
         sendToNode(dataMap.get(child.getId()), msgType, child);
@@ -327,8 +320,7 @@ public class OperatorTopologyStructImpl implements 
OperatorTopologyStruct {
         throw new RuntimeException("No message specified for " + child.getId() 
+ " in dataMap.");
       }
     }
-    LOG.exiting("OperatorTopologyStructImpl", "sendToChildren", 
Arrays.toString(new Object[]{getQualifiedName(),
-        dataMap, msgType}));
+    LOG.exiting("OperatorTopologyStructImpl", "sendToChildren", 
getQualifiedName());
   }
 
   @Override
@@ -336,8 +328,7 @@ public class OperatorTopologyStructImpl implements 
OperatorTopologyStruct {
     LOG.entering("OperatorTopologyStructImpl", "recvFromParent", 
getQualifiedName());
     LOG.finest(getQualifiedName() + "Waiting for " + parent.getId() + " to 
send data");
     final byte[] retVal = recvFromNodeCheckBigMsg(parent, msgType);
-    LOG.exiting("OperatorTopologyStructImpl", "recvFromParent",
-        Arrays.toString(new Object[]{retVal, getQualifiedName()}));
+    LOG.exiting("OperatorTopologyStructImpl", "recvFromParent", 
getQualifiedName());
     return retVal;
   }
 
@@ -367,8 +358,7 @@ public class OperatorTopologyStructImpl implements 
OperatorTopologyStruct {
       childrenToRcvFrom.remove(child.getId());
     }
     final T retVal = retLst.isEmpty() ? null : retLst.get(0);
-    LOG.exiting("OperatorTopologyStructImpl", "recvFromChildren",
-        Arrays.toString(new Object[]{retVal, getQualifiedName(), redFunc, 
dataCodec}));
+    LOG.exiting("OperatorTopologyStructImpl", "recvFromChildren", 
getQualifiedName());
     return retVal;
   }
 
@@ -399,8 +389,7 @@ public class OperatorTopologyStructImpl implements 
OperatorTopologyStruct {
       childrenToRcvFrom.remove(child.getId());
     }
 
-    LOG.exiting("OperatorTopologyStructImpl", "recvFromChildren",
-        Arrays.toString(new Object[]{retVal, getQualifiedName()}));
+    LOG.exiting("OperatorTopologyStructImpl", "recvFromChildren", 
getQualifiedName());
     return retVal;
   }
 
@@ -421,7 +410,7 @@ public class OperatorTopologyStructImpl implements 
OperatorTopologyStruct {
       LOG.finest(getQualifiedName() + "No dead msgs waiting for add.");
     }
     LOG.exiting("OperatorTopologyStructImpl", "removedDeadMsg",
-        Arrays.toString(new Object[]{retVal, getQualifiedName(), msgSrcId, 
msgSrcVersion}));
+        new Object[]{retVal, getQualifiedName(), msgSrcId, msgSrcVersion});
     return retVal;
   }
 

Reply via email to