-
Notifications
You must be signed in to change notification settings - Fork 751
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[GOBBLIN-1652]Add more log in the KafkaJobStatusMonitor in case it fails to process one GobblinTrackingEvent #3513
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -221,56 +221,63 @@ protected void processMessage(DecodeableKafkaRecord<byte[],byte[]> message) { | |
@VisibleForTesting | ||
static void addJobStatusToStateStore(org.apache.gobblin.configuration.State jobStatus, StateStore stateStore) | ||
throws IOException { | ||
if (!jobStatus.contains(TimingEvent.FlowEventConstants.JOB_NAME_FIELD)) { | ||
jobStatus.setProp(TimingEvent.FlowEventConstants.JOB_NAME_FIELD, JobStatusRetriever.NA_KEY); | ||
} | ||
if (!jobStatus.contains(TimingEvent.FlowEventConstants.JOB_GROUP_FIELD)) { | ||
jobStatus.setProp(TimingEvent.FlowEventConstants.JOB_GROUP_FIELD, JobStatusRetriever.NA_KEY); | ||
} | ||
String flowName = jobStatus.getProp(TimingEvent.FlowEventConstants.FLOW_NAME_FIELD); | ||
String flowGroup = jobStatus.getProp(TimingEvent.FlowEventConstants.FLOW_GROUP_FIELD); | ||
String flowExecutionId = jobStatus.getProp(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD); | ||
String jobName = jobStatus.getProp(TimingEvent.FlowEventConstants.JOB_NAME_FIELD); | ||
String jobGroup = jobStatus.getProp(TimingEvent.FlowEventConstants.JOB_GROUP_FIELD); | ||
String storeName = jobStatusStoreName(flowGroup, flowName); | ||
String tableName = jobStatusTableName(flowExecutionId, jobGroup, jobName); | ||
|
||
List<org.apache.gobblin.configuration.State> states = stateStore.getAll(storeName, tableName); | ||
if (states.size() > 0) { | ||
org.apache.gobblin.configuration.State previousJobStatus = states.get(states.size() - 1); | ||
String previousStatus = previousJobStatus.getProp(JobStatusRetriever.EVENT_NAME_FIELD); | ||
String currentStatus = jobStatus.getProp(JobStatusRetriever.EVENT_NAME_FIELD); | ||
int previousGeneration = previousJobStatus.getPropAsInt(TimingEvent.FlowEventConstants.CURRENT_GENERATION_FIELD, 1); | ||
// This is to make the change backward compatible as we may not have this info in cluster events | ||
// If we does not have those info, we treat the event as coming from the same attempts as previous one | ||
int currentGeneration = jobStatus.getPropAsInt(TimingEvent.FlowEventConstants.CURRENT_GENERATION_FIELD, previousGeneration); | ||
int previousAttempts = previousJobStatus.getPropAsInt(TimingEvent.FlowEventConstants.CURRENT_ATTEMPTS_FIELD, 1); | ||
int currentAttempts = jobStatus.getPropAsInt(TimingEvent.FlowEventConstants.CURRENT_ATTEMPTS_FIELD, previousAttempts); | ||
// We use three things to accurately count and thereby bound retries, even amidst out-of-order events (by skipping late arrivals). | ||
// The generation is monotonically increasing, while the attempts may re-initialize back to 0. this two-part form prevents the composite value from ever repeating. | ||
// And job status reflect the execution status in one attempt | ||
if (previousStatus != null && currentStatus != null && | ||
(previousGeneration > currentGeneration | ||
|| (previousGeneration == currentGeneration && previousAttempts > currentAttempts) | ||
|| (previousGeneration == currentGeneration && previousAttempts == currentAttempts | ||
&& ORDERED_EXECUTION_STATUSES.indexOf(ExecutionStatus.valueOf(currentStatus)) < ORDERED_EXECUTION_STATUSES.indexOf(ExecutionStatus.valueOf(previousStatus))))){ | ||
log.warn(String.format("Received status [generation.attempts] = %s [%s.%s] when already %s [%s.%s] for flow (%s, %s, %s), job (%s, %s)", | ||
currentStatus, currentGeneration, currentAttempts, previousStatus, previousGeneration, previousAttempts, flowGroup, flowName, flowExecutionId, jobGroup, jobName)); | ||
jobStatus = mergeState(states.get(states.size() - 1), jobStatus); | ||
} else { | ||
jobStatus = mergeState(jobStatus, states.get(states.size() - 1)); | ||
try { | ||
if (!jobStatus.contains(TimingEvent.FlowEventConstants.JOB_NAME_FIELD)) { | ||
jobStatus.setProp(TimingEvent.FlowEventConstants.JOB_NAME_FIELD, JobStatusRetriever.NA_KEY); | ||
} | ||
if (!jobStatus.contains(TimingEvent.FlowEventConstants.JOB_GROUP_FIELD)) { | ||
jobStatus.setProp(TimingEvent.FlowEventConstants.JOB_GROUP_FIELD, JobStatusRetriever.NA_KEY); | ||
} | ||
String flowName = jobStatus.getProp(TimingEvent.FlowEventConstants.FLOW_NAME_FIELD); | ||
String flowGroup = jobStatus.getProp(TimingEvent.FlowEventConstants.FLOW_GROUP_FIELD); | ||
String flowExecutionId = jobStatus.getProp(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD); | ||
String jobName = jobStatus.getProp(TimingEvent.FlowEventConstants.JOB_NAME_FIELD); | ||
String jobGroup = jobStatus.getProp(TimingEvent.FlowEventConstants.JOB_GROUP_FIELD); | ||
String storeName = jobStatusStoreName(flowGroup, flowName); | ||
String tableName = jobStatusTableName(flowExecutionId, jobGroup, jobName); | ||
|
||
List<org.apache.gobblin.configuration.State> states = stateStore.getAll(storeName, tableName); | ||
if (states.size() > 0) { | ||
org.apache.gobblin.configuration.State previousJobStatus = states.get(states.size() - 1); | ||
String previousStatus = previousJobStatus.getProp(JobStatusRetriever.EVENT_NAME_FIELD); | ||
String currentStatus = jobStatus.getProp(JobStatusRetriever.EVENT_NAME_FIELD); | ||
int previousGeneration = previousJobStatus.getPropAsInt(TimingEvent.FlowEventConstants.CURRENT_GENERATION_FIELD, 1); | ||
// This is to make the change backward compatible as we may not have this info in cluster events | ||
// If we does not have those info, we treat the event as coming from the same attempts as previous one | ||
int currentGeneration = jobStatus.getPropAsInt(TimingEvent.FlowEventConstants.CURRENT_GENERATION_FIELD, previousGeneration); | ||
int previousAttempts = previousJobStatus.getPropAsInt(TimingEvent.FlowEventConstants.CURRENT_ATTEMPTS_FIELD, 1); | ||
int currentAttempts = jobStatus.getPropAsInt(TimingEvent.FlowEventConstants.CURRENT_ATTEMPTS_FIELD, previousAttempts); | ||
// We use three things to accurately count and thereby bound retries, even amidst out-of-order events (by skipping late arrivals). | ||
// The generation is monotonically increasing, while the attempts may re-initialize back to 0. this two-part form prevents the composite value from ever repeating. | ||
// And job status reflect the execution status in one attempt | ||
if (previousStatus != null && currentStatus != null && (previousGeneration > currentGeneration || ( | ||
previousGeneration == currentGeneration && previousAttempts > currentAttempts) || (previousGeneration == currentGeneration && previousAttempts == currentAttempts | ||
&& ORDERED_EXECUTION_STATUSES.indexOf(ExecutionStatus.valueOf(currentStatus)) | ||
< ORDERED_EXECUTION_STATUSES.indexOf(ExecutionStatus.valueOf(previousStatus))))) { | ||
log.warn(String.format( | ||
"Received status [generation.attempts] = %s [%s.%s] when already %s [%s.%s] for flow (%s, %s, %s), job (%s, %s)", | ||
currentStatus, currentGeneration, currentAttempts, previousStatus, previousGeneration, previousAttempts, | ||
flowGroup, flowName, flowExecutionId, jobGroup, jobName)); | ||
jobStatus = mergeState(states.get(states.size() - 1), jobStatus); | ||
} else { | ||
jobStatus = mergeState(jobStatus, states.get(states.size() - 1)); | ||
} | ||
} | ||
} | ||
|
||
modifyStateIfRetryRequired(jobStatus); | ||
stateStore.put(storeName, tableName, jobStatus); | ||
modifyStateIfRetryRequired(jobStatus); | ||
stateStore.put(storeName, tableName, jobStatus); | ||
} catch (Exception e) { | ||
log.warn("Meet exception when adding jobStatus to state store at " | ||
+ e.getStackTrace()[0].getClassName() + "line number: " + e.getStackTrace()[0].getLineNumber(), e); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. won't the exception stack trace be logged, since you pass it as the final arg? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, based on what we noticed, if it's NPE, no much info will logged out, so I want to add the line info there in case we meet the same issue next time. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. makes sense to log that info. I was just thinking the full stack trace (
|
||
throw new IOException(e); | ||
} | ||
} | ||
|
||
private static void modifyStateIfRetryRequired(org.apache.gobblin.configuration.State state) { | ||
int maxAttempts = state.getPropAsInt(TimingEvent.FlowEventConstants.MAX_ATTEMPTS_FIELD, 1); | ||
int currentAttempts = state.getPropAsInt(TimingEvent.FlowEventConstants.CURRENT_ATTEMPTS_FIELD, 1); | ||
// SHOULD_RETRY_FIELD maybe reset by JOB_COMPLETION_PERCENTAGE event | ||
if ((state.getProp(JobStatusRetriever.EVENT_NAME_FIELD).equals(ExecutionStatus.FAILED.name()) | ||
if (state.contains(JobStatusRetriever.EVENT_NAME_FIELD) &&(state.getProp(JobStatusRetriever.EVENT_NAME_FIELD).equals(ExecutionStatus.FAILED.name()) | ||
|| state.getProp(JobStatusRetriever.EVENT_NAME_FIELD).equals(ExecutionStatus.PENDING_RETRY.name()) | ||
|| (state.getProp(JobStatusRetriever.EVENT_NAME_FIELD).equals(ExecutionStatus.CANCELLED.name()) && state.contains(TimingEvent.FlowEventConstants.DOES_CANCELED_FLOW_MERIT_RETRY)) | ||
) && currentAttempts < maxAttempts) { | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
what kind of exception were we seeing here? curious whether the entire block above needs wrapping or perhaps just the tail end subset of it
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We are seeing NPE without any information, so want to catch the entire method