Skip to content

Commit ba385cf

Browse files
committed
TEZ-4340: replace srcNameTrimmed in ShuffleManager and ShuffleScheduler
1 parent 0200dd9 commit ba385cf

File tree

5 files changed

+47
-42
lines changed

5 files changed

+47
-42
lines changed

tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java

Lines changed: 27 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -185,7 +185,7 @@ public class ShuffleManager implements FetcherCallback {
185185
*/
186186
private final int maxTimeToWaitForReportMillis;
187187

188-
private final String srcNameTrimmed;
188+
private final String sourceDestNameTrimmed;
189189

190190
private final int maxTaskOutputAtOnce;
191191

@@ -264,8 +264,9 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu
264264
conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS,
265265
TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS_DEFAULT);
266266

267-
this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName());
268-
267+
this.sourceDestNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()) + " -> "
268+
+ TezUtilsInternal.cleanVertexName(inputContext.getTaskVertexName());
269+
269270
completedInputSet = new BitSet(numInputs);
270271
/**
271272
* In case of pipelined shuffle, it is possible to get multiple FetchedInput per attempt.
@@ -288,15 +289,15 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu
288289
if (conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCHER_USE_SHARED_POOL,
289290
TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCHER_USE_SHARED_POOL_DEFAULT)) {
290291
fetcherRawExecutor = inputContext.createTezFrameworkExecutorService(numFetchers,
291-
"Fetcher_B {" + srcNameTrimmed + "} #%d");
292+
"Fetcher_B {" + sourceDestNameTrimmed + "} #%d");
292293
} else {
293294
fetcherRawExecutor = Executors.newFixedThreadPool(numFetchers, new ThreadFactoryBuilder()
294-
.setDaemon(true).setNameFormat("Fetcher_B {" + srcNameTrimmed + "} #%d").build());
295+
.setDaemon(true).setNameFormat("Fetcher_B {" + sourceDestNameTrimmed + "} #%d").build());
295296
}
296297
this.fetcherExecutor = MoreExecutors.listeningDecorator(fetcherRawExecutor);
297298

298299
ExecutorService schedulerRawExecutor = Executors.newFixedThreadPool(1, new ThreadFactoryBuilder()
299-
.setDaemon(true).setNameFormat("ShuffleRunner {" + srcNameTrimmed + "}").build());
300+
.setDaemon(true).setNameFormat("ShuffleRunner {" + sourceDestNameTrimmed + "}").build());
300301
this.schedulerExecutor = MoreExecutors.listeningDecorator(schedulerRawExecutor);
301302
this.schedulerCallable = new RunShuffleCallable(conf);
302303

@@ -336,7 +337,7 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu
336337

337338
shuffleInfoEventsMap = new ConcurrentHashMap<Integer, ShuffleEventInfo>();
338339

339-
LOG.info(srcNameTrimmed + ": numInputs=" + numInputs + ", compressionCodec="
340+
LOG.info(sourceDestNameTrimmed + ": numInputs=" + numInputs + ", compressionCodec="
340341
+ (codec == null ? "NoCompressionCodec" : codec.getClass().getName()) + ", numFetchers="
341342
+ numFetchers + ", ifileBufferSize=" + ifileBufferSize + ", ifileReadAheadEnabled="
342343
+ ifileReadAhead + ", ifileReadAheadLength=" + ifileReadAheadLength +", "
@@ -361,7 +362,7 @@ public void run() throws IOException {
361362
if (maxTimeToWaitForReportMillis > 0) {
362363
reporterExecutor = Executors.newSingleThreadExecutor(
363364
new ThreadFactoryBuilder().setDaemon(true)
364-
.setNameFormat("ShuffleRunner {" + srcNameTrimmed + "}")
365+
.setNameFormat("ShuffleRunner {" + sourceDestNameTrimmed + "}")
365366
.build());
366367
Future reporterFuture = reporterExecutor.submit(new ReporterCallable());
367368
}
@@ -446,7 +447,7 @@ protected Void callInternal() throws Exception {
446447
break;
447448
}
448449

449-
LOG.debug("{}: NumCompletedInputs: {}", srcNameTrimmed, numCompletedInputs);
450+
LOG.debug("{}: NumCompletedInputs: {}", sourceDestNameTrimmed, numCompletedInputs);
450451
if (numCompletedInputs.get() < numInputs && !isShutdown.get()) {
451452
lock.lock();
452453
try {
@@ -458,22 +459,22 @@ protected Void callInternal() throws Exception {
458459
inputHost = pendingHosts.take();
459460
} catch (InterruptedException e) {
460461
if (isShutdown.get()) {
461-
LOG.info(srcNameTrimmed + ": " + "Interrupted and hasBeenShutdown, Breaking out of ShuffleScheduler Loop");
462+
LOG.info(sourceDestNameTrimmed + ": " + "Interrupted and hasBeenShutdown, Breaking out of ShuffleScheduler Loop");
462463
Thread.currentThread().interrupt();
463464
break;
464465
} else {
465466
throw e;
466467
}
467468
}
468469
if (LOG.isDebugEnabled()) {
469-
LOG.debug(srcNameTrimmed + ": " + "Processing pending host: " +
470+
LOG.debug(sourceDestNameTrimmed + ": " + "Processing pending host: " +
470471
inputHost.toDetailedString());
471472
}
472473
if (inputHost.getNumPendingPartitions() > 0 && !isShutdown.get()) {
473474
Fetcher fetcher = constructFetcherForHost(inputHost, conf);
474475
runningFetchers.add(fetcher);
475476
if (isShutdown.get()) {
476-
LOG.info(srcNameTrimmed + ": " + "hasBeenShutdown," +
477+
LOG.info(sourceDestNameTrimmed + ": " + "hasBeenShutdown," +
477478
"Breaking out of ShuffleScheduler Loop");
478479
break;
479480
}
@@ -485,7 +486,7 @@ protected Void callInternal() throws Exception {
485486
}
486487
} else {
487488
if (LOG.isDebugEnabled()) {
488-
LOG.debug(srcNameTrimmed + ": " + "Skipping host: " +
489+
LOG.debug(sourceDestNameTrimmed + ": " + "Skipping host: " +
489490
inputHost.getIdentifier() +
490491
" since it has no inputs to process");
491492
}
@@ -497,7 +498,7 @@ protected Void callInternal() throws Exception {
497498
}
498499
}
499500
shufflePhaseTime.setValue(System.currentTimeMillis() - startTime);
500-
LOG.info(srcNameTrimmed + ": " + "Shutting down FetchScheduler, Was Interrupted: " + Thread.currentThread().isInterrupted());
501+
LOG.info(sourceDestNameTrimmed + ": " + "Shutting down FetchScheduler, Was Interrupted: " + Thread.currentThread().isInterrupted());
501502
if (!fetcherExecutor.isShutdown()) {
502503
fetcherExecutor.shutdownNow();
503504
}
@@ -540,13 +541,13 @@ Fetcher constructFetcherForHost(InputHost inputHost, Configuration conf) {
540541

541542
if (sharedFetchEnabled) {
542543
// pick a single lock disk from the edge name's hashcode + host hashcode
543-
final int h = Math.abs(Objects.hashCode(this.srcNameTrimmed, inputHost.getHost()));
544+
final int h = Math.abs(Objects.hashCode(this.sourceDestNameTrimmed, inputHost.getHost()));
544545
lockDisk = new Path(this.localDisks[h % this.localDisks.length], "locks");
545546
}
546547

547548
FetcherBuilder fetcherBuilder = new FetcherBuilder(ShuffleManager.this,
548549
httpConnectionParams, inputManager, inputContext.getApplicationId(), inputContext.getDagIdentifier(),
549-
jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator,
550+
jobTokenSecretMgr, sourceDestNameTrimmed, conf, localFs, localDirAllocator,
550551
lockDisk, localDiskFetchEnabled, sharedFetchEnabled,
551552
localhostName, shufflePort, asyncHttp, verifyDiskChecksum, compositeFetch, enableFetcherTestingErrors,
552553
inputContext.getObjectRegistry());
@@ -632,7 +633,7 @@ public void addKnownInput(String hostName, int port,
632633
}
633634
}
634635
if (LOG.isDebugEnabled()) {
635-
LOG.debug(srcNameTrimmed + ": " + "Adding input: " +
636+
LOG.debug(sourceDestNameTrimmed + ": " + "Adding input: " +
636637
srcAttemptIdentifier + ", to host: " + host);
637638
}
638639

@@ -950,7 +951,7 @@ public void fetchFailed(String host,
950951
LOG.info(
951952
"{}: Fetch failed for src: {} InputIdentifier: {}, connectFailed: {}, "
952953
+ "local fetch: {}, remote fetch failure reported as local failure: {})",
953-
srcNameTrimmed, srcAttemptIdentifier, srcAttemptIdentifier, connectFailed,
954+
sourceDestNameTrimmed, srcAttemptIdentifier, srcAttemptIdentifier, connectFailed,
954955
inputAttemptFetchFailure.isLocalFetch(), inputAttemptFetchFailure.isDiskErrorAtSource());
955956
failedShufflesCounter.increment(1);
956957
inputContext.notifyProgress();
@@ -988,11 +989,11 @@ public void shutdown() throws InterruptedException {
988989
if (Thread.currentThread().isInterrupted()) {
989990
//TODO: need to cleanup all FetchedInput (DiskFetchedInput, LocalDisFetchedInput), lockFile
990991
//As of now relying on job cleanup (when all directories would be cleared)
991-
LOG.info(srcNameTrimmed + ": " + "Thread interrupted. Need to cleanup the local dirs");
992+
LOG.info(sourceDestNameTrimmed + ": " + "Thread interrupted. Need to cleanup the local dirs");
992993
}
993994
if (!isShutdown.getAndSet(true)) {
994995
// Shut down any pending fetchers
995-
LOG.info("Shutting down pending fetchers on source" + srcNameTrimmed + ": "
996+
LOG.info("Shutting down pending fetchers on source" + sourceDestNameTrimmed + ": "
996997
+ runningFetchers.size());
997998
lock.lock();
998999
try {
@@ -1140,15 +1141,15 @@ private class SchedulerFutureCallback implements FutureCallback<Void> {
11401141

11411142
@Override
11421143
public void onSuccess(Void result) {
1143-
LOG.info(srcNameTrimmed + ": " + "Scheduler thread completed");
1144+
LOG.info(sourceDestNameTrimmed + ": " + "Scheduler thread completed");
11441145
}
11451146

11461147
@Override
11471148
public void onFailure(Throwable t) {
11481149
if (isShutdown.get()) {
1149-
LOG.debug("{}: Already shutdown. Ignoring error.", srcNameTrimmed, t);
1150+
LOG.debug("{}: Already shutdown. Ignoring error.", sourceDestNameTrimmed, t);
11501151
} else {
1151-
LOG.error(srcNameTrimmed + ": " + "Scheduler failed with error: ", t);
1152+
LOG.error(sourceDestNameTrimmed + ": " + "Scheduler failed with error: ", t);
11521153
inputContext.reportFailure(TaskFailureType.NON_FATAL, t, "Shuffle Scheduler Failed");
11531154
}
11541155
}
@@ -1177,7 +1178,7 @@ private void doBookKeepingForFetcherComplete() {
11771178
public void onSuccess(FetchResult result) {
11781179
fetcher.shutdown();
11791180
if (isShutdown.get()) {
1180-
LOG.debug("{}: Already shutdown. Ignoring event from fetcher", srcNameTrimmed);
1181+
LOG.debug("{}: Already shutdown. Ignoring event from fetcher", sourceDestNameTrimmed);
11811182
} else {
11821183
Iterable<InputAttemptIdentifier> pendingInputs = result.getPendingInputs();
11831184
if (pendingInputs != null && pendingInputs.iterator().hasNext()) {
@@ -1200,9 +1201,9 @@ public void onFailure(Throwable t) {
12001201
// Unsuccessful - the fetcher may not have shutdown correctly. Try shutting it down.
12011202
fetcher.shutdown();
12021203
if (isShutdown.get()) {
1203-
LOG.debug("{}: Already shutdown. Ignoring error from fetcher.", srcNameTrimmed, t);
1204+
LOG.debug("{}: Already shutdown. Ignoring error from fetcher.", sourceDestNameTrimmed, t);
12041205
} else {
1205-
LOG.error(srcNameTrimmed + ": " + "Fetcher failed with error: ", t);
1206+
LOG.error(sourceDestNameTrimmed + ": " + "Fetcher failed with error: ", t);
12061207
shuffleError = t;
12071208
inputContext.reportFailure(TaskFailureType.NON_FATAL, t, "Fetch failed");
12081209
doBookKeepingForFetcherComplete();

tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java

Lines changed: 17 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -93,7 +93,7 @@ public class Shuffle implements ExceptionReporter {
9393
private volatile ListenableFuture<TezRawKeyValueIterator> runShuffleFuture;
9494
private final ListeningExecutorService executor;
9595

96-
private final String srcNameTrimmed;
96+
private final String sourceDestNameTrimmed;
9797

9898
private AtomicBoolean isShutDown = new AtomicBoolean(false);
9999
private AtomicBoolean fetchersClosed = new AtomicBoolean(false);
@@ -109,7 +109,8 @@ public Shuffle(InputContext inputContext, Configuration conf, int numInputs,
109109
this.inputContext = inputContext;
110110
this.conf = conf;
111111

112-
this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName());
112+
this.sourceDestNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()) + " -> "
113+
+ TezUtilsInternal.cleanVertexName(inputContext.getTaskVertexName());
113114

114115
this.codec = CodecUtils.getCodec(conf);
115116

@@ -138,7 +139,7 @@ public Shuffle(InputContext inputContext, Configuration conf, int numInputs,
138139
TezCounter mergedMapOutputsCounter =
139140
inputContext.getCounters().findCounter(TaskCounter.MERGED_MAP_OUTPUTS);
140141

141-
LOG.info(srcNameTrimmed + ": " + "Shuffle assigned with " + numInputs + " inputs" + ", codec: "
142+
LOG.info(sourceDestNameTrimmed + ": " + "Shuffle assigned with " + numInputs + " inputs" + ", codec: "
142143
+ (codec == null ? "None" : codec.getClass().getName())
143144
+ ", ifileReadAhead: " + ifileReadAhead);
144145

@@ -169,7 +170,7 @@ public Shuffle(InputContext inputContext, Configuration conf, int numInputs,
169170
codec,
170171
ifileReadAhead,
171172
ifileReadAheadLength,
172-
srcNameTrimmed);
173+
sourceDestNameTrimmed);
173174

174175
this.mergePhaseTime = inputContext.getCounters().findCounter(TaskCounter.MERGE_PHASE_TIME);
175176
this.shufflePhaseTime = inputContext.getCounters().findCounter(TaskCounter.SHUFFLE_PHASE_TIME);
@@ -182,7 +183,7 @@ public Shuffle(InputContext inputContext, Configuration conf, int numInputs,
182183
ShuffleUtils.isTezShuffleHandler(conf));
183184

184185
ExecutorService rawExecutor = Executors.newFixedThreadPool(1, new ThreadFactoryBuilder()
185-
.setDaemon(true).setNameFormat("ShuffleAndMergeRunner {" + srcNameTrimmed + "}").build());
186+
.setDaemon(true).setNameFormat("ShuffleAndMergeRunner {" + sourceDestNameTrimmed + "}").build());
186187

187188

188189
executor = MoreExecutors.listeningDecorator(rawExecutor);
@@ -193,7 +194,7 @@ public void handleEvents(List<Event> events) throws IOException {
193194
if (!isShutDown.get()) {
194195
eventHandler.handleEvents(events);
195196
} else {
196-
LOG.info(srcNameTrimmed + ": " + "Ignoring events since already shutdown. EventCount: " + events.size());
197+
LOG.info(sourceDestNameTrimmed + ": " + "Ignoring events since already shutdown. EventCount: " + events.size());
197198
}
198199

199200
}
@@ -267,7 +268,7 @@ public void run() throws IOException {
267268
public void shutdown() {
268269
if (!isShutDown.getAndSet(true)) {
269270
// Interrupt so that the scheduler / merger sees this interrupt.
270-
LOG.info("Shutting down Shuffle for source: " + srcNameTrimmed);
271+
LOG.info("Shutting down Shuffle for source: " + sourceDestNameTrimmed);
271272
runShuffleFuture.cancel(true);
272273
cleanupIgnoreErrors();
273274
}
@@ -323,7 +324,7 @@ protected TezRawKeyValueIterator callInternal() throws IOException, InterruptedE
323324
}
324325

325326
inputContext.inputIsReady();
326-
LOG.info("merge complete for input vertex : " + srcNameTrimmed);
327+
LOG.info("merge complete for input vertex : " + sourceDestNameTrimmed);
327328
return kvIter;
328329
}
329330
}
@@ -333,7 +334,7 @@ private void cleanupShuffleSchedulerIgnoreErrors() {
333334
cleanupShuffleScheduler();
334335
} catch (InterruptedException e) {
335336
Thread.currentThread().interrupt();
336-
LOG.info(srcNameTrimmed + ": " + "Interrupted while attempting to close the scheduler during cleanup. Ignoring");
337+
LOG.info(sourceDestNameTrimmed + ": " + "Interrupted while attempting to close the scheduler during cleanup. Ignoring");
337338
}
338339
}
339340

@@ -351,13 +352,13 @@ private void cleanupMerger(boolean ignoreErrors) throws Throwable {
351352
if (ignoreErrors) {
352353
//Reset the status
353354
Thread.currentThread().interrupt();
354-
LOG.info(srcNameTrimmed + ": " + "Interrupted while attempting to close the merger during cleanup. Ignoring");
355+
LOG.info(sourceDestNameTrimmed + ": " + "Interrupted while attempting to close the merger during cleanup. Ignoring");
355356
} else {
356357
throw e;
357358
}
358359
} catch (Throwable e) {
359360
if (ignoreErrors) {
360-
LOG.info(srcNameTrimmed + ": " + "Exception while trying to shutdown merger, Ignoring", e);
361+
LOG.info(sourceDestNameTrimmed + ": " + "Exception while trying to shutdown merger, Ignoring", e);
361362
} else {
362363
throw e;
363364
}
@@ -379,7 +380,7 @@ private void cleanupIgnoreErrors() {
379380
}
380381
cleanupMerger(true);
381382
} catch (Throwable t) {
382-
LOG.info(srcNameTrimmed + ": " + "Error in cleaning up.., ", t);
383+
LOG.info(sourceDestNameTrimmed + ": " + "Error in cleaning up.., ", t);
383384
}
384385
}
385386

@@ -388,7 +389,7 @@ private void cleanupIgnoreErrors() {
388389
public synchronized void reportException(Throwable t) {
389390
// RunShuffleCallable onFailure deals with ignoring errors on shutdown.
390391
if (throwable.get() == null) {
391-
LOG.info(srcNameTrimmed + ": " + "Setting throwable in reportException with message [" + t.getMessage() +
392+
LOG.info(sourceDestNameTrimmed + ": " + "Setting throwable in reportException with message [" + t.getMessage() +
392393
"] from thread [" + Thread.currentThread().getName());
393394
throwable.set(t);
394395
throwingThreadName = Thread.currentThread().getName();
@@ -423,15 +424,15 @@ public static long getInitialMemoryRequirement(Configuration conf, long maxAvail
423424
private class ShuffleRunnerFutureCallback implements FutureCallback<TezRawKeyValueIterator> {
424425
@Override
425426
public void onSuccess(TezRawKeyValueIterator result) {
426-
LOG.info(srcNameTrimmed + ": " + "Shuffle Runner thread complete");
427+
LOG.info(sourceDestNameTrimmed + ": " + "Shuffle Runner thread complete");
427428
}
428429

429430
@Override
430431
public void onFailure(Throwable t) {
431432
if (isShutDown.get()) {
432-
LOG.info(srcNameTrimmed + ": " + "Already shutdown. Ignoring error");
433+
LOG.info(sourceDestNameTrimmed + ": " + "Already shutdown. Ignoring error");
433434
} else {
434-
LOG.error(srcNameTrimmed + ": " + "ShuffleRunner failed with error", t);
435+
LOG.error(sourceDestNameTrimmed + ": " + "ShuffleRunner failed with error", t);
435436
// In case of an abort / Interrupt - the runtime makes sure that this is ignored.
436437
inputContext.reportFailure(TaskFailureType.NON_FATAL, t, "Shuffle Runner Failed");
437438
cleanupIgnoreErrors();

tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -189,6 +189,7 @@ private InputContext createInputContext() throws IOException {
189189
InputContext inputContext = mock(InputContext.class);
190190
doReturn(new TezCounters()).when(inputContext).getCounters();
191191
doReturn("sourceVertex").when(inputContext).getSourceVertexName();
192+
doReturn("taskVertex").when(inputContext).getTaskVertexName();
192193
doReturn(shuffleMetaData).when(inputContext)
193194
.getServiceProviderMetaData(conf.get(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID,
194195
TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID_DEFAULT));

tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -169,6 +169,7 @@ private InputContext createInputContext() throws IOException {
169169
InputContext inputContext = mock(InputContext.class);
170170
doReturn(new TezCounters()).when(inputContext).getCounters();
171171
doReturn("sourceVertex").when(inputContext).getSourceVertexName();
172+
doReturn("taskVertex").when(inputContext).getTaskVertexName();
172173
doReturn(shuffleMetaData).when(inputContext)
173174
.getServiceProviderMetaData(conf.get(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID,
174175
TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID_DEFAULT));

tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -144,6 +144,7 @@ private InputContext createTezInputContext() throws IOException {
144144
InputContext inputContext = mock(InputContext.class);
145145
doReturn(applicationId).when(inputContext).getApplicationId();
146146
doReturn("sourceVertex").when(inputContext).getSourceVertexName();
147+
doReturn("taskVertex").when(inputContext).getTaskVertexName();
147148
when(inputContext.getCounters()).thenReturn(new TezCounters());
148149
ExecutionContext executionContext = new ExecutionContextImpl("localhost");
149150
doReturn(executionContext).when(inputContext).getExecutionContext();

0 commit comments

Comments
 (0)