Skip to content

Commit cc8249e

Browse files
authored
TEZ-4340: Show convenient input -> output vertex names in input messages (#170) (Csaba Juhasz reviewed by Laszlo Bodor)
1 parent 0b3f3b6 commit cc8249e

23 files changed

Lines changed: 209 additions & 191 deletions

File tree

tez-api/src/main/java/org/apache/tez/runtime/api/InputContext.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,12 @@ public interface InputContext extends TaskContext {
3232
* @return Name of the Source Vertex
3333
*/
3434
public String getSourceVertexName();
35+
36+
/**
37+
* Returns a convenient, human-readable string describing the input and output vertices.
38+
* @return the convenient string
39+
*/
40+
String getInputOutputVertexNames();
3541

3642
/**
3743
* Get the index of the input in the set of all inputs for the task. The

tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -461,7 +461,7 @@ public List<Event> initialize() throws IOException {
461461
getContext().inputIsReady();
462462
this.splitInfoViaEvents = jobConf.getBoolean(MRJobConfig.MR_TEZ_SPLITS_VIA_EVENTS,
463463
MRJobConfig.MR_TEZ_SPLITS_VIA_EVENTS_DEFAULT);
464-
LOG.info(getContext().getSourceVertexName() + " using newmapreduce API=" + useNewApi +
464+
LOG.info(getContext().getInputOutputVertexNames() + " using newmapreduce API=" + useNewApi +
465465
", split via event=" + splitInfoViaEvents + ", numPhysicalInputs=" +
466466
getNumPhysicalInputs());
467467
initializeInternal();
@@ -526,7 +526,7 @@ inputRecordCounter, getContext().getApplicationId().getClusterTimestamp(),
526526
} finally {
527527
rrLock.unlock();
528528
}
529-
LOG.info("Initialized MRInput: " + getContext().getSourceVertexName());
529+
LOG.info("Initialized MRInput: " + getContext().getInputOutputVertexNames());
530530
}
531531

532532
/**
@@ -634,7 +634,7 @@ void processSplitEvent(InputDataInformationEvent event)
634634
try {
635635
initFromEventInternal(event);
636636
if (LOG.isDebugEnabled()) {
637-
LOG.debug(getContext().getSourceVertexName() + " notifying on RecordReader initialized");
637+
LOG.debug(getContext().getInputOutputVertexNames() + " notifying on RecordReader initialized");
638638
}
639639
rrInited.signal();
640640
} finally {
@@ -647,7 +647,7 @@ void checkAndAwaitRecordReaderInitialization() throws IOException {
647647
rrLock.lock();
648648
try {
649649
if (LOG.isDebugEnabled()) {
650-
LOG.debug(getContext().getSourceVertexName() + " awaiting RecordReader initialization");
650+
LOG.debug(getContext().getInputOutputVertexNames() + " awaiting RecordReader initialization");
651651
}
652652
rrInited.await();
653653
} catch (Exception e) {
@@ -671,7 +671,7 @@ void initFromEvent(InputDataInformationEvent initEvent)
671671

672672
private void initFromEventInternal(InputDataInformationEvent initEvent) throws IOException {
673673
if (LOG.isDebugEnabled()) {
674-
LOG.debug(getContext().getSourceVertexName() + " initializing RecordReader from event");
674+
LOG.debug(getContext().getInputOutputVertexNames() + " initializing RecordReader from event");
675675
}
676676
Objects.requireNonNull(initEvent, "InitEvent must be specified");
677677
MRSplitProto splitProto = MRSplitProto.parseFrom(ByteString.copyFrom(initEvent.getUserPayload()));
@@ -686,7 +686,7 @@ private void initFromEventInternal(InputDataInformationEvent initEvent) throws I
686686
LOG.warn("Thread interrupted while getting split length: ", e);
687687
}
688688
if (LOG.isDebugEnabled()) {
689-
LOG.debug(getContext().getSourceVertexName() + " split Details -> SplitClass: " +
689+
LOG.debug(getContext().getInputOutputVertexNames() + " split Details -> SplitClass: " +
690690
split.getClass().getName() + ", NewSplit: " + split + ", length: " + splitLength);
691691
}
692692

@@ -696,7 +696,7 @@ private void initFromEventInternal(InputDataInformationEvent initEvent) throws I
696696
splitObj = split;
697697
splitLength = split.getLength();
698698
if (LOG.isDebugEnabled()) {
699-
LOG.debug(getContext().getSourceVertexName() + " split Details -> SplitClass: " +
699+
LOG.debug(getContext().getInputOutputVertexNames() + " split Details -> SplitClass: " +
700700
split.getClass().getName() + ", OldSplit: " + split + ", length: " + splitLength);
701701
}
702702
}
@@ -705,7 +705,7 @@ private void initFromEventInternal(InputDataInformationEvent initEvent) throws I
705705
.increment(splitLength);
706706
}
707707
mrReader.setSplit(splitObj);
708-
LOG.info(getContext().getSourceVertexName() + " initialized RecordReader from event");
708+
LOG.info(getContext().getInputOutputVertexNames() + " initialized RecordReader from event");
709709
}
710710

711711
private static class MRInputHelpersInternal extends MRInputHelpers {

tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInputLegacy.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -74,7 +74,7 @@ public MRInputLegacy(InputContext inputContext, int numPhysicalInputs) {
7474

7575
@Private
7676
protected void initializeInternal() throws IOException {
77-
LOG.info(getContext().getSourceVertexName() + " MRInputLegacy deferring initialization");
77+
LOG.info(getContext().getInputOutputVertexNames() + " MRInputLegacy deferring initialization");
7878
}
7979

8080
@Private
@@ -136,7 +136,7 @@ void checkAndAwaitRecordReaderInitialization() throws IOException {
136136
if (splitInfoViaEvents && !inited) {
137137
if (initEvent == null) {
138138
if (LOG.isDebugEnabled()) {
139-
LOG.debug(getContext().getSourceVertexName() +
139+
LOG.debug(getContext().getInputOutputVertexNames() +
140140
" awaiting init event before initializing record reader");
141141
}
142142

tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -110,7 +110,7 @@ private MultiMRInputConfigBuilder(Configuration conf, Class<?> inputFormat) {
110110
@Override
111111
public List<Event> initialize() throws IOException {
112112
super.initialize();
113-
LOG.info(getContext().getSourceVertexName() + " using newmapreduce API=" + useNewApi +
113+
LOG.info(getContext().getInputOutputVertexNames() + " using newmapreduce API=" + useNewApi +
114114
", numPhysicalInputs=" + getNumPhysicalInputs());
115115
if (getNumPhysicalInputs() == 0) {
116116
getContext().inputIsReady();
@@ -167,7 +167,7 @@ public void handleEvents(List<Event> inputEvents) throws Exception {
167167
private MRReader initFromEvent(InputDataInformationEvent event) throws IOException {
168168
Objects.requireNonNull(event, "Event must be specified");
169169
if (LOG.isDebugEnabled()) {
170-
LOG.debug(getContext().getSourceVertexName() + " initializing Reader: " + eventCount.get());
170+
LOG.debug(getContext().getInputOutputVertexNames() + " initializing Reader: " + eventCount.get());
171171
}
172172
MRSplitProto splitProto = MRSplitProto.parseFrom(ByteString.copyFrom(event.getUserPayload()));
173173
MRReader reader = null;
@@ -186,7 +186,7 @@ private MRReader initFromEvent(InputDataInformationEvent event) throws IOExcepti
186186
.getApplicationId().getId(), getContext().getTaskIndex(), getContext()
187187
.getTaskAttemptNumber(), getContext());
188188
if (LOG.isDebugEnabled()) {
189-
LOG.debug(getContext().getSourceVertexName() + " split Details -> SplitClass: " +
189+
LOG.debug(getContext().getInputOutputVertexNames() + " split Details -> SplitClass: " +
190190
split.getClass().getName() + ", NewSplit: " + split + ", length: " + splitLength);
191191
}
192192
} else {
@@ -196,15 +196,15 @@ private MRReader initFromEvent(InputDataInformationEvent event) throws IOExcepti
196196
reader = new MRReaderMapred(localJobConf, split,
197197
getContext().getCounters(), inputRecordCounter, getContext());
198198
if (LOG.isDebugEnabled()) {
199-
LOG.debug(getContext().getSourceVertexName() + " split Details -> SplitClass: " +
199+
LOG.debug(getContext().getInputOutputVertexNames() + " split Details -> SplitClass: " +
200200
split.getClass().getName() + ", OldSplit: " + split + ", length: " + splitLength);
201201
}
202202
}
203203
if (splitLength != -1) {
204204
getContext().getCounters().findCounter(TaskCounter.INPUT_SPLIT_LENGTH_BYTES)
205205
.increment(splitLength);
206206
}
207-
LOG.info(getContext().getSourceVertexName() + " initialized RecordReader from event");
207+
LOG.info(getContext().getInputOutputVertexNames() + " initialized RecordReader from event");
208208
return reader;
209209
}
210210

tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -152,6 +152,11 @@ public String getSourceVertexName() {
152152
return sourceVertexName;
153153
}
154154

155+
@Override
156+
public String getInputOutputVertexNames() {
157+
return String.format("%s -> %s", getSourceVertexName(), getTaskVertexName());
158+
}
159+
155160
@Override
156161
public void fatalError(Throwable exception, String message) {
157162
super.signalFatalError(exception, message, sourceInfo);

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

Lines changed: 23 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -58,7 +58,7 @@
5858
import org.apache.tez.common.CallableWithNdc;
5959
import org.apache.tez.common.security.JobTokenSecretManager;
6060
import org.apache.tez.dag.api.TezUncheckedException;
61-
import org.apache.tez.runtime.api.ObjectRegistry;
61+
import org.apache.tez.runtime.api.InputContext;
6262
import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
6363
import org.apache.tez.runtime.library.common.Constants;
6464
import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
@@ -69,6 +69,7 @@
6969
import org.apache.tez.runtime.library.common.shuffle.FetchedInput.Type;
7070
import org.apache.tez.runtime.library.common.shuffle.api.ShuffleHandlerError;
7171
import org.apache.tez.common.Preconditions;
72+
import org.apache.tez.common.TezUtilsInternal;
7273

7374
/**
7475
* Responsible for fetching inputs served by the ShuffleHandler for a single
@@ -194,8 +195,8 @@ public String getHost() {
194195
private final boolean isDebugEnabled = LOG.isDebugEnabled();
195196

196197
protected Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params,
197-
FetchedInputAllocator inputManager, ApplicationId appId, int dagIdentifier,
198-
JobTokenSecretManager jobTokenSecretManager, String srcNameTrimmed, Configuration conf,
198+
FetchedInputAllocator inputManager, InputContext inputContext,
199+
JobTokenSecretManager jobTokenSecretManager, Configuration conf,
199200
RawLocalFileSystem localFs,
200201
LocalDirAllocator localDirAllocator,
201202
Path lockPath,
@@ -208,8 +209,8 @@ protected Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params,
208209
this.fetcherCallback = fetcherCallback;
209210
this.inputManager = inputManager;
210211
this.jobTokenSecretMgr = jobTokenSecretManager;
211-
this.appId = appId;
212-
this.dagIdentifier = dagIdentifier;
212+
this.appId = inputContext.getApplicationId();
213+
this.dagIdentifier = inputContext.getDagIdentifier();
213214
this.pathToAttemptMap = new HashMap<PathPartition, InputAttemptIdentifier>();
214215
this.httpConnectionParams = params;
215216
this.conf = conf;
@@ -218,7 +219,10 @@ protected Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params,
218219
this.sharedFetchEnabled = sharedFetchEnabled;
219220

220221
this.fetcherIdentifier = fetcherIdGen.getAndIncrement();
221-
this.logIdentifier = " fetcher [" + srcNameTrimmed +"] " + fetcherIdentifier;
222+
223+
String sourceDestNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()) + " -> "
224+
+ TezUtilsInternal.cleanVertexName(inputContext.getTaskVertexName());
225+
this.logIdentifier = " fetcher [" + sourceDestNameTrimmed +"] " + fetcherIdentifier;
222226

223227
this.localFs = localFs;
224228
this.localDirAllocator = localDirAllocator;
@@ -1133,31 +1137,29 @@ public static class FetcherBuilder {
11331137
private boolean workAssigned = false;
11341138

11351139
public FetcherBuilder(FetcherCallback fetcherCallback,
1136-
HttpConnectionParams params, FetchedInputAllocator inputManager,
1137-
ApplicationId appId, int dagIdentifier, JobTokenSecretManager jobTokenSecretMgr, String srcNameTrimmed,
1138-
Configuration conf, boolean localDiskFetchEnabled, String localHostname, int shufflePort,
1139-
boolean asyncHttp, boolean verifyDiskChecksum, boolean compositeFetch) {
1140-
this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId, dagIdentifier,
1141-
jobTokenSecretMgr, srcNameTrimmed, conf, null, null, null, localDiskFetchEnabled,
1140+
HttpConnectionParams params, FetchedInputAllocator inputManager, InputContext inputContext,
1141+
JobTokenSecretManager jobTokenSecretMgr, Configuration conf, boolean localDiskFetchEnabled,
1142+
String localHostname, int shufflePort, boolean asyncHttp, boolean verifyDiskChecksum, boolean compositeFetch) {
1143+
this.fetcher = new Fetcher(fetcherCallback, params, inputManager, inputContext,
1144+
jobTokenSecretMgr, conf, null, null, null, localDiskFetchEnabled,
11421145
false, localHostname, shufflePort, asyncHttp, verifyDiskChecksum, compositeFetch);
11431146
}
11441147

11451148
public FetcherBuilder(FetcherCallback fetcherCallback,
1146-
HttpConnectionParams params, FetchedInputAllocator inputManager,
1147-
ApplicationId appId, int dagIdentifier, JobTokenSecretManager jobTokenSecretMgr, String srcNameTrimmed,
1148-
Configuration conf, RawLocalFileSystem localFs,
1149+
HttpConnectionParams params, FetchedInputAllocator inputManager, InputContext inputContext,
1150+
JobTokenSecretManager jobTokenSecretMgr, Configuration conf, RawLocalFileSystem localFs,
11491151
LocalDirAllocator localDirAllocator, Path lockPath,
11501152
boolean localDiskFetchEnabled, boolean sharedFetchEnabled,
11511153
String localHostname, int shufflePort, boolean asyncHttp, boolean verifyDiskChecksum, boolean compositeFetch,
1152-
boolean enableFetcherTestingErrors, ObjectRegistry objectRegistry) {
1154+
boolean enableFetcherTestingErrors) {
11531155
if (enableFetcherTestingErrors) {
1154-
this.fetcher = new FetcherWithInjectableErrors(fetcherCallback, params, inputManager, appId, dagIdentifier,
1155-
jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator,
1156+
this.fetcher = new FetcherWithInjectableErrors(fetcherCallback, params, inputManager, inputContext,
1157+
jobTokenSecretMgr, conf, localFs, localDirAllocator,
11561158
lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, asyncHttp,
1157-
verifyDiskChecksum, compositeFetch, objectRegistry);
1159+
verifyDiskChecksum, compositeFetch);
11581160
} else {
1159-
this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId, dagIdentifier,
1160-
jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator,
1161+
this.fetcher = new Fetcher(fetcherCallback, params, inputManager, inputContext,
1162+
jobTokenSecretMgr, conf, localFs, localDirAllocator,
11611163
lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, asyncHttp,
11621164
verifyDiskChecksum, compositeFetch);
11631165
}

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

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -20,10 +20,10 @@
2020
import org.apache.hadoop.fs.LocalDirAllocator;
2121
import org.apache.hadoop.fs.Path;
2222
import org.apache.hadoop.fs.RawLocalFileSystem;
23-
import org.apache.hadoop.yarn.api.records.ApplicationId;
23+
import org.apache.tez.common.TezUtilsInternal;
2424
import org.apache.tez.common.security.JobTokenSecretManager;
2525
import org.apache.tez.http.HttpConnectionParams;
26-
import org.apache.tez.runtime.api.ObjectRegistry;
26+
import org.apache.tez.runtime.api.InputContext;
2727
import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
2828
import org.slf4j.Logger;
2929
import org.slf4j.LoggerFactory;
@@ -35,16 +35,16 @@ public class FetcherWithInjectableErrors extends Fetcher {
3535
private String srcNameTrimmed;
3636

3737
protected FetcherWithInjectableErrors(FetcherCallback fetcherCallback, HttpConnectionParams params,
38-
FetchedInputAllocator inputManager, ApplicationId appId, int dagIdentifier,
39-
JobTokenSecretManager jobTokenSecretManager, String srcNameTrimmed, Configuration conf,
38+
FetchedInputAllocator inputManager, InputContext inputContext,
39+
JobTokenSecretManager jobTokenSecretManager, Configuration conf,
4040
RawLocalFileSystem localFs, LocalDirAllocator localDirAllocator, Path lockPath, boolean localDiskFetchEnabled,
4141
boolean sharedFetchEnabled, String localHostname, int shufflePort, boolean asyncHttp, boolean verifyDiskChecksum,
42-
boolean compositeFetch, ObjectRegistry objectRegistry) {
43-
super(fetcherCallback, params, inputManager, appId, dagIdentifier, jobTokenSecretManager, srcNameTrimmed, conf,
42+
boolean compositeFetch) {
43+
super(fetcherCallback, params, inputManager, inputContext, jobTokenSecretManager, conf,
4444
localFs, localDirAllocator, lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort,
4545
asyncHttp, verifyDiskChecksum, compositeFetch);
46-
this.fetcherErrorTestingConfig = new FetcherErrorTestingConfig(conf, objectRegistry);
47-
this.srcNameTrimmed = srcNameTrimmed;
46+
this.fetcherErrorTestingConfig = new FetcherErrorTestingConfig(conf, inputContext.getObjectRegistry());
47+
this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName());
4848
LOG.info("Initialized FetcherWithInjectableErrors with config: {}", fetcherErrorTestingConfig);
4949
}
5050

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -161,7 +161,7 @@ private void handleEvent(Event event) throws IOException {
161161

162162
@Override
163163
public void logProgress(boolean updateOnClose) {
164-
LOG.info(inputContext.getSourceVertexName() + ": "
164+
LOG.info(inputContext.getInputOutputVertexNames() + ": "
165165
+ "numDmeEventsSeen=" + numDmeEvents.get()
166166
+ ", numDmeEventsSeenWithNoData=" + numDmeEventsNoData.get()
167167
+ ", numObsoletionEventsSeen=" + numObsoletionEvents.get()

0 commit comments

Comments
 (0)