@@ -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 ();
0 commit comments