You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by pa...@apache.org on 2014/06/08 23:04:13 UTC
git commit: updated refs/heads/trunk to e4b8db5
Repository: giraph
Updated Branches:
refs/heads/trunk 0c345c6a7 -> e4b8db5d8
GIRAPH-900: Remove timers for compute method (majakabiljo via pavanka)
Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/e4b8db5d
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/e4b8db5d
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/e4b8db5d
Branch: refs/heads/trunk
Commit: e4b8db5d8b34b33d7551d912e53fe198bb95d7e0
Parents: 0c345c6
Author: Pavan Kumar <pa...@fb.com>
Authored: Sun Jun 8 14:00:18 2014 -0700
Committer: Pavan Kumar <pa...@fb.com>
Committed: Sun Jun 8 14:00:18 2014 -0700
----------------------------------------------------------------------
CHANGELOG | 2 +
.../apache/giraph/graph/ComputeCallable.java | 15 +---
.../giraph/metrics/AggregatedMetrics.java | 7 --
.../giraph/metrics/GiraphMetricsRegistry.java | 12 ----
.../org/apache/giraph/metrics/TimerDesc.java | 72 --------------------
.../giraph/metrics/WorkerSuperstepMetrics.java | 19 ------
6 files changed, 3 insertions(+), 124 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/giraph/blob/e4b8db5d/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index dc87f48..fce8fb5 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
Giraph Change Log
Release 1.1.0 - unreleased
+ GIRAPH-900: Remove timers for compute method (majakabiljo via pavanka)
+
GIRAPH-894: make giraphconfiguration available to workercontext (pavanka)
GIRAPH-910: removing unnecessary boxing in some places in giraph-core (pavanka)
http://git-wip-us.apache.org/repos/asf/giraph/blob/e4b8db5d/giraph-core/src/main/java/org/apache/giraph/graph/ComputeCallable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/ComputeCallable.java b/giraph-core/src/main/java/org/apache/giraph/graph/ComputeCallable.java
index 86a1a52..d9c4302 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/ComputeCallable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/ComputeCallable.java
@@ -26,7 +26,6 @@ import org.apache.giraph.io.SimpleVertexWriter;
import org.apache.giraph.metrics.GiraphMetrics;
import org.apache.giraph.metrics.MetricNames;
import org.apache.giraph.metrics.SuperstepMetricsRegistry;
-import org.apache.giraph.metrics.TimerDesc;
import org.apache.giraph.partition.Partition;
import org.apache.giraph.partition.PartitionStats;
import org.apache.giraph.time.SystemTime;
@@ -46,8 +45,6 @@ import org.apache.log4j.Logger;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.yammer.metrics.core.Counter;
-import com.yammer.metrics.core.Timer;
-import com.yammer.metrics.core.TimerContext;
import java.io.IOException;
import java.util.Collection;
@@ -102,8 +99,6 @@ public class ComputeCallable<I extends WritableComparable, V extends Writable,
private final Counter messagesSentCounter;
/** Message bytes sent */
private final Counter messageBytesSentCounter;
- /** Timer for single compute() call */
- private final Timer computeOneTimer;
/**
* Constructor
@@ -129,9 +124,6 @@ public class ComputeCallable<I extends WritableComparable, V extends Writable,
this.graphState = graphState;
SuperstepMetricsRegistry metrics = GiraphMetrics.get().perSuperstep();
- // Normally we would use ResetSuperstepMetricsObserver but this class is
- // not long-lived, so just instantiating in the constructor is good enough.
- computeOneTimer = metrics.getTimer(TimerDesc.COMPUTE_ONE);
messagesSentCounter = metrics.getCounter(MetricNames.MESSAGES_SENT);
messageBytesSentCounter =
metrics.getCounter(MetricNames.MESSAGE_BYTES_SENT);
@@ -243,12 +235,7 @@ public class ComputeCallable<I extends WritableComparable, V extends Writable,
}
if (!vertex.isHalted()) {
context.progress();
- TimerContext computeOneTimerContext = computeOneTimer.time();
- try {
- computation.compute(vertex, messages);
- } finally {
- computeOneTimerContext.stop();
- }
+ computation.compute(vertex, messages);
// Need to unwrap the mutated edges (possibly)
vertex.unwrapMutableEdges();
//Compact edges representation if possible
http://git-wip-us.apache.org/repos/asf/giraph/blob/e4b8db5d/giraph-core/src/main/java/org/apache/giraph/metrics/AggregatedMetrics.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/metrics/AggregatedMetrics.java b/giraph-core/src/main/java/org/apache/giraph/metrics/AggregatedMetrics.java
index 83681e5..9d8d48f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/metrics/AggregatedMetrics.java
+++ b/giraph-core/src/main/java/org/apache/giraph/metrics/AggregatedMetrics.java
@@ -30,9 +30,6 @@ import java.util.Map;
* Map of a bunch of aggregated metrics
*/
public class AggregatedMetrics {
- /** counter for user compute time in milliseconds */
- public static final String USER_COMPUTE_MS = "user-compute-ms";
-
/** Mapping from name to aggregated metric */
private Map<String, AggregatedMetric> metrics = Maps.newHashMap();
@@ -74,8 +71,6 @@ public class AggregatedMetrics {
workerMetrics.getTimeToFirstMsg(), hostname);
add(BspServiceWorker.TIMER_WAIT_REQUESTS,
workerMetrics.getWaitRequestsTimer(), hostname);
- add(USER_COMPUTE_MS,
- workerMetrics.getUserComputeTime(), hostname);
return this;
}
@@ -94,13 +89,11 @@ public class AggregatedMetrics {
get(GraphTaskManager.TIMER_TIME_TO_FIRST_MSG);
AggregatedMetric waitRequestsMicros = get(
BspServiceWorker.TIMER_WAIT_REQUESTS);
- AggregatedMetric userComputeTime = get(USER_COMPUTE_MS);
out.println();
out.println("--- METRICS: superstep " + superstep + " ---");
printAggregatedMetric(out, "superstep time", "ms", superstepTime);
printAggregatedMetric(out, "compute all partitions", "ms", computeAll);
- printAggregatedMetric(out, "user compute time", "ms", userComputeTime);
printAggregatedMetric(out, "network communication time", "ms", commTime);
printAggregatedMetric(out, "time to first message", "us", timeToFirstMsg);
printAggregatedMetric(out, "wait requests time", "us", waitRequestsMicros);
http://git-wip-us.apache.org/repos/asf/giraph/blob/e4b8db5d/giraph-core/src/main/java/org/apache/giraph/metrics/GiraphMetricsRegistry.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/metrics/GiraphMetricsRegistry.java b/giraph-core/src/main/java/org/apache/giraph/metrics/GiraphMetricsRegistry.java
index 149f352..cb97fbe 100644
--- a/giraph-core/src/main/java/org/apache/giraph/metrics/GiraphMetricsRegistry.java
+++ b/giraph-core/src/main/java/org/apache/giraph/metrics/GiraphMetricsRegistry.java
@@ -258,18 +258,6 @@ public class GiraphMetricsRegistry {
}
/**
- * Create a new {@link Timer} from the description and registers it under the
- * given group and name.
- *
- * @param timerDesc TimerDesc describing the timer
- * @return new {@link Timer}
- */
- public Timer getTimer(TimerDesc timerDesc) {
- return getTimer(timerDesc.getName(), timerDesc.getDurationUnit(),
- timerDesc.getTimeUnit());
- }
-
- /**
* Creates a new {@link Timer} and registers it under the given
* group and name.
*
http://git-wip-us.apache.org/repos/asf/giraph/blob/e4b8db5d/giraph-core/src/main/java/org/apache/giraph/metrics/TimerDesc.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/metrics/TimerDesc.java b/giraph-core/src/main/java/org/apache/giraph/metrics/TimerDesc.java
deleted file mode 100644
index 1e7f3a9..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/metrics/TimerDesc.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.giraph.metrics;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * Description for Timers used in Giraph
- */
-public enum TimerDesc {
-
- /** Timer around Vertex#compute() */
- COMPUTE_ONE("compute-one", TimeUnit.MILLISECONDS, TimeUnit.SECONDS);
-
- /** Name of timer */
- private final String name;
- /** Duration unit for timer */
- private final TimeUnit durationUnit;
- /** Time unit for timer */
- private final TimeUnit timeUnit;
-
- /**
- * Constructor
- * @param name String name of timer
- * @param durationUnit Duration unit of timer
- * @param timeUnit Time unit of timer
- */
- private TimerDesc(String name, TimeUnit durationUnit, TimeUnit timeUnit) {
- this.name = name;
- this.durationUnit = durationUnit;
- this.timeUnit = timeUnit;
- }
-
- /**
- * Get duration unit of timer
- * @return TimeUnit
- */
- public TimeUnit getDurationUnit() {
- return durationUnit;
- }
-
- /**
- * Get name of timer
- * @return String name
- */
- public String getName() {
- return name;
- }
-
- /**
- * Get time unit of timer
- * @return TimeUnit of timer
- */
- public TimeUnit getTimeUnit() {
- return timeUnit;
- }
-}
http://git-wip-us.apache.org/repos/asf/giraph/blob/e4b8db5d/giraph-core/src/main/java/org/apache/giraph/metrics/WorkerSuperstepMetrics.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/metrics/WorkerSuperstepMetrics.java b/giraph-core/src/main/java/org/apache/giraph/metrics/WorkerSuperstepMetrics.java
index e51f96e..05ec55b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/metrics/WorkerSuperstepMetrics.java
+++ b/giraph-core/src/main/java/org/apache/giraph/metrics/WorkerSuperstepMetrics.java
@@ -28,7 +28,6 @@ import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.io.PrintStream;
-import java.util.concurrent.TimeUnit;
/**
* Per-superstep metrics for a Worker.
@@ -44,8 +43,6 @@ public class WorkerSuperstepMetrics implements Writable {
private LongAndTimeUnit superstepTimer;
/** Time spent waiting for other workers to finish */
private LongAndTimeUnit waitRequestsTimer;
- /** Time spent in Vertex#compute */
- private LongAndTimeUnit userComputeTime;
/**
* Constructor
@@ -56,10 +53,6 @@ public class WorkerSuperstepMetrics implements Writable {
timeToFirstMsg = new LongAndTimeUnit();
superstepTimer = new LongAndTimeUnit();
waitRequestsTimer = new LongAndTimeUnit();
-
- // Note this one is not backed by a GiraphTimer, but rather a real Timer
- userComputeTime = new LongAndTimeUnit();
- userComputeTime.setTimeUnit(TimeUnit.MILLISECONDS);
}
/**
@@ -68,13 +61,11 @@ public class WorkerSuperstepMetrics implements Writable {
* @return this object, for chaining
*/
public WorkerSuperstepMetrics readFromRegistry() {
- SuperstepMetricsRegistry ssm = GiraphMetrics.get().perSuperstep();
readGiraphTimer(GraphTaskManager.TIMER_COMMUNICATION_TIME, commTimer);
readGiraphTimer(GraphTaskManager.TIMER_COMPUTE_ALL, computeAllTimer);
readGiraphTimer(GraphTaskManager.TIMER_TIME_TO_FIRST_MSG, timeToFirstMsg);
readGiraphTimer(GraphTaskManager.TIMER_SUPERSTEP_TIME, superstepTimer);
readGiraphTimer(BspServiceWorker.TIMER_WAIT_REQUESTS, waitRequestsTimer);
- userComputeTime.setValue((long) ssm.getTimer(TimerDesc.COMPUTE_ONE).sum());
return this;
}
@@ -108,7 +99,6 @@ public class WorkerSuperstepMetrics implements Writable {
out.println("--- METRICS: superstep " + superstep + " ---");
out.println(" superstep time: " + superstepTimer);
out.println(" compute all partitions: " + computeAllTimer);
- out.println(" user compute time: " + userComputeTime);
out.println(" network communication time: " + commTimer);
out.println(" time to first message: " + timeToFirstMsg);
out.println(" wait on requests time: " + waitRequestsTimer);
@@ -150,13 +140,6 @@ public class WorkerSuperstepMetrics implements Writable {
return waitRequestsTimer.getValue();
}
- /**
- * @return milliseconds in user compute code
- */
- public long getUserComputeTime() {
- return userComputeTime.getValue();
- }
-
@Override
public void readFields(DataInput dataInput) throws IOException {
commTimer.setValue(dataInput.readLong());
@@ -164,7 +147,6 @@ public class WorkerSuperstepMetrics implements Writable {
timeToFirstMsg.setValue(dataInput.readLong());
superstepTimer.setValue(dataInput.readLong());
waitRequestsTimer.setValue(dataInput.readLong());
- userComputeTime.setValue(dataInput.readLong());
}
@Override
@@ -174,6 +156,5 @@ public class WorkerSuperstepMetrics implements Writable {
dataOutput.writeLong(timeToFirstMsg.getValue());
dataOutput.writeLong(superstepTimer.getValue());
dataOutput.writeLong(waitRequestsTimer.getValue());
- dataOutput.writeLong(userComputeTime.getValue());
}
}