1 | |
|
2 | |
|
3 | |
|
4 | |
|
5 | |
|
6 | |
|
7 | |
|
8 | |
|
9 | |
|
10 | |
|
11 | |
|
12 | |
|
13 | |
|
14 | |
|
15 | |
|
16 | |
|
17 | |
|
18 | |
package org.apache.giraph.graph; |
19 | |
|
20 | |
import java.io.IOException; |
21 | |
import java.util.Collection; |
22 | |
import java.util.List; |
23 | |
import java.util.concurrent.Callable; |
24 | |
|
25 | |
import org.apache.giraph.bsp.CentralizedServiceWorker; |
26 | |
import org.apache.giraph.comm.WorkerClientRequestProcessor; |
27 | |
import org.apache.giraph.comm.messages.MessageStore; |
28 | |
import org.apache.giraph.comm.netty.NettyWorkerClientRequestProcessor; |
29 | |
import org.apache.giraph.conf.GiraphConstants; |
30 | |
import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration; |
31 | |
import org.apache.giraph.function.primitive.PrimitiveRefs.LongRef; |
32 | |
import org.apache.giraph.io.SimpleVertexWriter; |
33 | |
import org.apache.giraph.metrics.GiraphMetrics; |
34 | |
import org.apache.giraph.metrics.MetricNames; |
35 | |
import org.apache.giraph.metrics.SuperstepMetricsRegistry; |
36 | |
import org.apache.giraph.ooc.OutOfCoreEngine; |
37 | |
import org.apache.giraph.partition.Partition; |
38 | |
import org.apache.giraph.partition.PartitionStats; |
39 | |
import org.apache.giraph.partition.PartitionStore; |
40 | |
import org.apache.giraph.time.SystemTime; |
41 | |
import org.apache.giraph.time.Time; |
42 | |
import org.apache.giraph.time.Times; |
43 | |
import org.apache.giraph.utils.MemoryUtils; |
44 | |
import org.apache.giraph.utils.TimedLogger; |
45 | |
import org.apache.giraph.utils.Trimmable; |
46 | |
import org.apache.giraph.worker.WorkerProgress; |
47 | |
import org.apache.giraph.worker.WorkerThreadGlobalCommUsage; |
48 | |
import org.apache.hadoop.io.Writable; |
49 | |
import org.apache.hadoop.io.WritableComparable; |
50 | |
import org.apache.hadoop.mapreduce.Mapper; |
51 | |
import org.apache.hadoop.util.Progressable; |
52 | |
import org.apache.log4j.Logger; |
53 | |
|
54 | |
import com.google.common.base.Preconditions; |
55 | |
import com.google.common.collect.Iterables; |
56 | |
import com.google.common.collect.Lists; |
57 | |
import com.yammer.metrics.core.Counter; |
58 | |
import com.yammer.metrics.core.Histogram; |
59 | |
|
60 | |
|
61 | |
|
62 | |
|
63 | |
|
64 | |
|
65 | |
|
66 | |
|
67 | |
|
68 | |
|
69 | |
|
70 | |
|
71 | |
|
72 | |
|
73 | |
|
74 | 0 | public class ComputeCallable<I extends WritableComparable, V extends Writable, |
75 | |
E extends Writable, M1 extends Writable, M2 extends Writable> |
76 | |
implements Callable<Collection<PartitionStats>> { |
77 | |
|
78 | 0 | private static final Logger LOG = Logger.getLogger(ComputeCallable.class); |
79 | |
|
80 | 0 | private static final Time TIME = SystemTime.get(); |
81 | |
|
82 | |
private final long verticesToUpdateProgress; |
83 | |
|
84 | |
private final Mapper<?, ?, ?, ?>.Context context; |
85 | |
|
86 | |
private final GraphState graphState; |
87 | |
|
88 | |
private final MessageStore<I, M1> messageStore; |
89 | |
|
90 | |
private final ImmutableClassesGiraphConfiguration<I, V, E> configuration; |
91 | |
|
92 | |
private final CentralizedServiceWorker<I, V, E> serviceWorker; |
93 | |
|
94 | 0 | private final TimedLogger timedLogger = new TimedLogger(30 * 1000, LOG); |
95 | |
|
96 | |
private SimpleVertexWriter<I, V, E> vertexWriter; |
97 | |
|
98 | 0 | private final long startNanos = TIME.getNanoseconds(); |
99 | |
|
100 | |
|
101 | |
|
102 | |
private final Counter messagesSentCounter; |
103 | |
|
104 | |
private final Counter messageBytesSentCounter; |
105 | |
|
106 | |
private final Histogram histogramComputePerPartition; |
107 | |
|
108 | |
private final Histogram histogramGCTimePerThread; |
109 | |
|
110 | |
private final Histogram histogramWaitTimePerThread; |
111 | |
|
112 | |
private final Histogram histogramProcessingTimePerThread; |
113 | |
|
114 | |
|
115 | |
|
116 | |
|
117 | |
|
118 | |
|
119 | |
|
120 | |
|
121 | |
|
122 | |
|
123 | |
public ComputeCallable(Mapper<?, ?, ?, ?>.Context context, |
124 | |
GraphState graphState, MessageStore<I, M1> messageStore, |
125 | |
ImmutableClassesGiraphConfiguration<I, V, E> configuration, |
126 | 0 | CentralizedServiceWorker<I, V, E> serviceWorker) { |
127 | 0 | this.context = context; |
128 | 0 | this.configuration = configuration; |
129 | 0 | this.messageStore = messageStore; |
130 | 0 | this.serviceWorker = serviceWorker; |
131 | 0 | this.graphState = graphState; |
132 | |
|
133 | 0 | SuperstepMetricsRegistry metrics = GiraphMetrics.get().perSuperstep(); |
134 | 0 | messagesSentCounter = metrics.getCounter(MetricNames.MESSAGES_SENT); |
135 | 0 | messageBytesSentCounter = |
136 | 0 | metrics.getCounter(MetricNames.MESSAGE_BYTES_SENT); |
137 | 0 | histogramComputePerPartition = metrics.getUniformHistogram( |
138 | |
MetricNames.HISTOGRAM_COMPUTE_PER_PARTITION); |
139 | 0 | histogramGCTimePerThread = metrics.getUniformHistogram("gc-per-thread-ms"); |
140 | 0 | histogramWaitTimePerThread = |
141 | 0 | metrics.getUniformHistogram("wait-per-thread-ms"); |
142 | 0 | histogramProcessingTimePerThread = |
143 | 0 | metrics.getUniformHistogram("processing-per-thread-ms"); |
144 | 0 | verticesToUpdateProgress = |
145 | 0 | GiraphConstants.VERTICES_TO_UPDATE_PROGRESS.get(configuration); |
146 | 0 | } |
147 | |
|
148 | |
@Override |
149 | |
public Collection<PartitionStats> call() { |
150 | |
|
151 | 0 | WorkerClientRequestProcessor<I, V, E> workerClientRequestProcessor = |
152 | |
new NettyWorkerClientRequestProcessor<I, V, E>( |
153 | |
context, configuration, serviceWorker, |
154 | 0 | configuration.getOutgoingMessageEncodeAndStoreType(). |
155 | 0 | useOneMessageToManyIdsEncoding()); |
156 | 0 | WorkerThreadGlobalCommUsage aggregatorUsage = |
157 | 0 | serviceWorker.getAggregatorHandler().newThreadAggregatorUsage(); |
158 | |
|
159 | 0 | vertexWriter = serviceWorker.getSuperstepOutput().getVertexWriter(); |
160 | |
|
161 | 0 | Computation<I, V, E, M1, M2> computation = |
162 | 0 | (Computation<I, V, E, M1, M2>) configuration.createComputation(); |
163 | 0 | computation.initialize(graphState, workerClientRequestProcessor, |
164 | |
serviceWorker, aggregatorUsage); |
165 | 0 | computation.preSuperstep(); |
166 | |
|
167 | 0 | List<PartitionStats> partitionStatsList = Lists.newArrayList(); |
168 | 0 | PartitionStore<I, V, E> partitionStore = serviceWorker.getPartitionStore(); |
169 | 0 | OutOfCoreEngine oocEngine = serviceWorker.getServerData().getOocEngine(); |
170 | 0 | GraphTaskManager<I, V, E> taskManager = serviceWorker.getGraphTaskManager(); |
171 | 0 | if (oocEngine != null) { |
172 | 0 | oocEngine.processingThreadStart(); |
173 | |
} |
174 | 0 | long timeWaiting = 0; |
175 | 0 | long timeProcessing = 0; |
176 | 0 | long timeDoingGC = 0; |
177 | |
while (true) { |
178 | 0 | long startTime = System.currentTimeMillis(); |
179 | 0 | long startGCTime = taskManager.getSuperstepGCTime(); |
180 | 0 | Partition<I, V, E> partition = partitionStore.getNextPartition(); |
181 | 0 | long timeDoingGCWhileWaiting = |
182 | 0 | taskManager.getSuperstepGCTime() - startGCTime; |
183 | 0 | timeDoingGC += timeDoingGCWhileWaiting; |
184 | 0 | timeWaiting += System.currentTimeMillis() - startTime - |
185 | |
timeDoingGCWhileWaiting; |
186 | 0 | if (partition == null) { |
187 | 0 | break; |
188 | |
} |
189 | 0 | long startProcessingTime = System.currentTimeMillis(); |
190 | 0 | startGCTime = taskManager.getSuperstepGCTime(); |
191 | |
try { |
192 | 0 | serviceWorker.getServerData().resolvePartitionMutation(partition); |
193 | 0 | PartitionStats partitionStats = computePartition( |
194 | |
computation, partition, oocEngine, |
195 | 0 | serviceWorker.getConfiguration().getIncomingMessageClasses() |
196 | 0 | .ignoreExistingVertices()); |
197 | 0 | partitionStatsList.add(partitionStats); |
198 | 0 | long partitionMsgs = workerClientRequestProcessor.resetMessageCount(); |
199 | 0 | partitionStats.addMessagesSentCount(partitionMsgs); |
200 | 0 | messagesSentCounter.inc(partitionMsgs); |
201 | 0 | long partitionMsgBytes = |
202 | 0 | workerClientRequestProcessor.resetMessageBytesCount(); |
203 | 0 | partitionStats.addMessageBytesSentCount(partitionMsgBytes); |
204 | 0 | messageBytesSentCounter.inc(partitionMsgBytes); |
205 | 0 | timedLogger.info("call: Completed " + |
206 | 0 | partitionStatsList.size() + " partitions, " + |
207 | 0 | partitionStore.getNumPartitions() + " remaining " + |
208 | 0 | MemoryUtils.getRuntimeMemoryStats()); |
209 | 0 | long timeDoingGCWhileProcessing = |
210 | 0 | taskManager.getSuperstepGCTime() - startGCTime; |
211 | 0 | timeDoingGC += timeDoingGCWhileProcessing; |
212 | |
long timeProcessingPartition = |
213 | 0 | System.currentTimeMillis() - startProcessingTime - |
214 | |
timeDoingGCWhileProcessing; |
215 | 0 | timeProcessing += timeProcessingPartition; |
216 | 0 | partitionStats.setComputeMs(timeProcessingPartition); |
217 | 0 | } catch (IOException e) { |
218 | 0 | throw new IllegalStateException("call: Caught unexpected IOException," + |
219 | |
" failing.", e); |
220 | 0 | } catch (InterruptedException e) { |
221 | 0 | throw new IllegalStateException("call: Caught unexpected " + |
222 | |
"InterruptedException, failing.", e); |
223 | |
} finally { |
224 | 0 | partitionStore.putPartition(partition); |
225 | 0 | } |
226 | 0 | histogramComputePerPartition.update( |
227 | 0 | System.currentTimeMillis() - startTime); |
228 | 0 | } |
229 | 0 | histogramGCTimePerThread.update(timeDoingGC); |
230 | 0 | histogramWaitTimePerThread.update(timeWaiting); |
231 | 0 | histogramProcessingTimePerThread.update(timeProcessing); |
232 | 0 | computation.postSuperstep(); |
233 | |
|
234 | |
|
235 | 0 | serviceWorker.getSuperstepOutput().returnVertexWriter(vertexWriter); |
236 | |
|
237 | 0 | if (LOG.isInfoEnabled()) { |
238 | 0 | float seconds = Times.getNanosSince(TIME, startNanos) / |
239 | |
Time.NS_PER_SECOND_AS_FLOAT; |
240 | 0 | LOG.info("call: Computation took " + seconds + " secs for " + |
241 | 0 | partitionStatsList.size() + " partitions on superstep " + |
242 | 0 | graphState.getSuperstep() + ". Flushing started (time waiting on " + |
243 | |
"partitions was " + |
244 | 0 | String.format("%.2f s", timeWaiting / 1000.0) + ", time processing " + |
245 | 0 | "partitions was " + String.format("%.2f s", timeProcessing / 1000.0) + |
246 | |
", time spent on gc was " + |
247 | 0 | String.format("%.2f s", timeDoingGC / 1000.0) + ")"); |
248 | |
} |
249 | |
try { |
250 | 0 | workerClientRequestProcessor.flush(); |
251 | |
|
252 | |
|
253 | 0 | if (partitionStatsList.size() > 0) { |
254 | 0 | long partitionMsgBytes = |
255 | 0 | workerClientRequestProcessor.resetMessageBytesCount(); |
256 | 0 | partitionStatsList.get(partitionStatsList.size() - 1). |
257 | 0 | addMessageBytesSentCount(partitionMsgBytes); |
258 | 0 | messageBytesSentCounter.inc(partitionMsgBytes); |
259 | |
} |
260 | 0 | aggregatorUsage.finishThreadComputation(); |
261 | 0 | } catch (IOException e) { |
262 | 0 | throw new IllegalStateException("call: Flushing failed.", e); |
263 | 0 | } |
264 | 0 | if (oocEngine != null) { |
265 | 0 | oocEngine.processingThreadFinish(); |
266 | |
} |
267 | 0 | return partitionStatsList; |
268 | |
} |
269 | |
|
270 | |
|
271 | |
|
272 | |
|
273 | |
|
274 | |
|
275 | |
|
276 | |
|
277 | |
|
278 | |
|
279 | |
private PartitionStats computePartition( |
280 | |
Computation<I, V, E, M1, M2> computation, |
281 | |
Partition<I, V, E> partition, OutOfCoreEngine oocEngine, |
282 | |
boolean ignoreExistingVertices) |
283 | |
throws IOException, InterruptedException { |
284 | 0 | PartitionStats partitionStats = |
285 | 0 | new PartitionStats(partition.getId(), 0, 0, 0, 0, 0, |
286 | 0 | serviceWorker.getWorkerInfo().getHostnameId()); |
287 | 0 | final LongRef verticesComputedProgress = new LongRef(0); |
288 | |
|
289 | 0 | Progressable verticesProgressable = new Progressable() { |
290 | |
@Override |
291 | |
public void progress() { |
292 | 0 | verticesComputedProgress.value++; |
293 | 0 | if (verticesComputedProgress.value == verticesToUpdateProgress) { |
294 | 0 | WorkerProgress.get().addVerticesComputed( |
295 | |
verticesComputedProgress.value); |
296 | 0 | verticesComputedProgress.value = 0; |
297 | |
} |
298 | 0 | } |
299 | |
}; |
300 | |
|
301 | 0 | synchronized (partition) { |
302 | 0 | if (ignoreExistingVertices) { |
303 | 0 | Iterable<I> destinations = |
304 | 0 | messageStore.getPartitionDestinationVertices(partition.getId()); |
305 | 0 | if (!Iterables.isEmpty(destinations)) { |
306 | 0 | OnlyIdVertex<I> vertex = new OnlyIdVertex<>(); |
307 | |
|
308 | 0 | for (I vertexId : destinations) { |
309 | 0 | Iterable<M1> messages = messageStore.getVertexMessages(vertexId); |
310 | 0 | Preconditions.checkState(!Iterables.isEmpty(messages)); |
311 | 0 | vertex.setId(vertexId); |
312 | 0 | computation.compute((Vertex) vertex, messages); |
313 | |
|
314 | |
|
315 | 0 | messageStore.clearVertexMessages(vertexId); |
316 | |
|
317 | |
|
318 | 0 | partitionStats.incrVertexCount(); |
319 | |
|
320 | 0 | verticesProgressable.progress(); |
321 | 0 | } |
322 | |
} |
323 | 0 | } else { |
324 | 0 | int count = 0; |
325 | 0 | for (Vertex<I, V, E> vertex : partition) { |
326 | |
|
327 | |
|
328 | |
|
329 | 0 | if (oocEngine != null && |
330 | |
(++count & OutOfCoreEngine.CHECK_IN_INTERVAL) == 0) { |
331 | 0 | oocEngine.activeThreadCheckIn(); |
332 | |
} |
333 | 0 | Iterable<M1> messages = |
334 | 0 | messageStore.getVertexMessages(vertex.getId()); |
335 | 0 | if (vertex.isHalted() && !Iterables.isEmpty(messages)) { |
336 | 0 | vertex.wakeUp(); |
337 | |
} |
338 | 0 | if (!vertex.isHalted()) { |
339 | 0 | context.progress(); |
340 | 0 | computation.compute(vertex, messages); |
341 | |
|
342 | 0 | vertex.unwrapMutableEdges(); |
343 | |
|
344 | 0 | if (vertex instanceof Trimmable) { |
345 | 0 | ((Trimmable) vertex).trim(); |
346 | |
} |
347 | |
|
348 | 0 | vertexWriter.writeVertex(vertex); |
349 | |
|
350 | 0 | partition.saveVertex(vertex); |
351 | |
} |
352 | 0 | if (vertex.isHalted()) { |
353 | 0 | partitionStats.incrFinishedVertexCount(); |
354 | |
} |
355 | |
|
356 | 0 | messageStore.clearVertexMessages(vertex.getId()); |
357 | |
|
358 | |
|
359 | 0 | partitionStats.incrVertexCount(); |
360 | 0 | partitionStats.addEdgeCount(vertex.getNumEdges()); |
361 | |
|
362 | 0 | verticesProgressable.progress(); |
363 | 0 | } |
364 | |
} |
365 | 0 | messageStore.clearPartition(partition.getId()); |
366 | 0 | } |
367 | 0 | WorkerProgress.get().addVerticesComputed(verticesComputedProgress.value); |
368 | 0 | WorkerProgress.get().incrementPartitionsComputed(); |
369 | 0 | return partitionStats; |
370 | |
} |
371 | |
} |
372 | |
|