/
FailureEnricher.java
154 lines (139 loc) · 5.53 KB
/
FailureEnricher.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.core.failure;
import org.apache.flink.annotation.Experimental;
import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobInfo;
import org.apache.flink.metrics.MetricGroup;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executor;
/**
* Failure Enricher enabling custom logic and attaching metadata in the form of labels to each type
* of failure as tracked in the JobMaster.
*/
@Experimental
public interface FailureEnricher {
/**
* Method to list all the label Keys the enricher can associate with Values in case of a failure
* {@code processFailure}. Note that Keys must unique and properly defined per enricher
* implementation otherwise will be ignored.
*
* @return the unique label Keys of the FailureEnricher
*/
Set<String> getOutputKeys();
/**
* Method to handle a failure as part of the enricher and optionally return a map of KV pairs
* (labels). Note that Values should only be associated with Keys from {@code getOutputKeys}
* method otherwise will be ignored.
*
* @param cause the exception that caused this failure
* @param context the context that includes extra information (e.g., if it was a global failure)
* @return map of KV pairs (labels) associated with the failure
*/
CompletableFuture<Map<String, String>> processFailure(
final Throwable cause, final Context context);
/**
* An interface used by the {@link FailureEnricher}. Context includes an executor pool for the
* enrichers to run heavy operations, the Classloader used for code gen, and other metadata.
*/
@Experimental
interface Context {
/** Type of failure. */
enum FailureType {
/**
* The failure has occurred in the scheduler context and can't be tracked back to a
* particular task.
*/
GLOBAL,
/** The failure has been reported by a particular task. */
TASK,
/**
* The TaskManager has non-gracefully disconnected from the JobMaster or we have not
* received heartbeats for the {@link
* org.apache.flink.configuration.HeartbeatManagerOptions#HEARTBEAT_INTERVAL configured
* timeout}.
*/
TASK_MANAGER
}
/**
* Get the ID of the job.
*
* @return the ID of the job
* @deprecated This method is deprecated since Flink 1.19. All metadata about the job should
* be provided uniformly by {@link #getJobInfo()}.
* @see <a
* href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-382%3A+Unify+the+Provision+of+Diverse+Metadata+for+Context-like+APIs">
* FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs </a>
*/
@Deprecated
default JobID getJobId() {
return getJobInfo().getJobId();
}
/**
* Get the name of the job.
*
* @return the name of the job
* @deprecated This method is deprecated since Flink 1.19. All metadata about the job should
* be provided uniformly by {@link #getJobInfo()}.
* @see <a
* href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-382%3A+Unify+the+Provision+of+Diverse+Metadata+for+Context-like+APIs">
* FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs </a>
*/
@Deprecated
default String getJobName() {
return getJobInfo().getJobName();
}
/**
* Get the metric group of the JobMaster.
*
* @return the metric group of the JobMaster
*/
MetricGroup getMetricGroup();
/**
* Return the type of the failure e.g., global failure that happened in the scheduler
* context.
*
* @return FailureType
*/
FailureType getFailureType();
/**
* Get the user {@link ClassLoader} used for code generation, UDF loading and other
* operations requiring reflections on user code.
*
* @return the user ClassLoader
*/
ClassLoader getUserClassLoader();
/**
* Get an Executor pool for the Enrichers to run async operations that can potentially be
* IO-heavy.
*
* @return the Executor pool
*/
Executor getIOExecutor();
/**
* Get the meta information of current job.
*
* @return the job meta information.
*/
@PublicEvolving
JobInfo getJobInfo();
}
}