18
18
19
19
package org .apache .flink .kubernetes .operator .metrics ;
20
20
21
+ import org .apache .flink .annotation .VisibleForTesting ;
21
22
import org .apache .flink .kubernetes .operator .config .FlinkOperatorConfiguration ;
22
23
import org .apache .flink .kubernetes .operator .metrics .OperatorMetricUtils .SynchronizedMeterView ;
23
24
import org .apache .flink .metrics .Counter ;
24
25
import org .apache .flink .metrics .Histogram ;
25
26
import org .apache .flink .metrics .MeterView ;
26
27
import org .apache .flink .metrics .MetricGroup ;
27
28
28
- import okhttp3 .Interceptor ;
29
- import okhttp3 .Request ;
30
- import okhttp3 .Response ;
29
+ import io .fabric8 .kubernetes .client .http .AsyncBody ;
30
+ import io .fabric8 .kubernetes .client .http .BasicBuilder ;
31
+ import io .fabric8 .kubernetes .client .http .HttpRequest ;
32
+ import io .fabric8 .kubernetes .client .http .HttpResponse ;
33
+ import io .fabric8 .kubernetes .client .http .Interceptor ;
34
+ import org .slf4j .Logger ;
35
+ import org .slf4j .LoggerFactory ;
31
36
32
- import java .io . IOException ;
37
+ import java .nio . ByteBuffer ;
33
38
import java .util .ArrayList ;
34
39
import java .util .List ;
35
40
import java .util .Map ;
41
+ import java .util .concurrent .CompletableFuture ;
36
42
import java .util .concurrent .ConcurrentHashMap ;
43
+ import java .util .function .LongSupplier ;
37
44
38
45
/** Kubernetes client metrics. */
39
46
public class KubernetesClientMetrics implements Interceptor {
@@ -50,6 +57,7 @@ public class KubernetesClientMetrics implements Interceptor {
50
57
public static final String COUNTER = "Count" ;
51
58
public static final String METER = "NumPerSecond" ;
52
59
public static final String HISTO = "TimeNanos" ;
60
+ public static final String REQUEST_START_TIME_HEADER = "requestStartTimeNanos" ;
53
61
private final Histogram responseLatency ;
54
62
55
63
private final MetricGroup requestMetricGroup ;
@@ -69,9 +77,20 @@ public class KubernetesClientMetrics implements Interceptor {
69
77
private final Map <Integer , SynchronizedMeterView > responseCodeMeters =
70
78
new ConcurrentHashMap <>();
71
79
private final Map <String , Counter > requestMethodCounter = new ConcurrentHashMap <>();
80
+ private final LongSupplier nanoTimeSource ;
81
+
82
+ private final Logger logger = LoggerFactory .getLogger (KubernetesClientMetrics .class );
72
83
73
84
public KubernetesClientMetrics (
74
85
MetricGroup parentGroup , FlinkOperatorConfiguration flinkOperatorConfiguration ) {
86
+ this (parentGroup , flinkOperatorConfiguration , System ::nanoTime );
87
+ }
88
+
89
+ public KubernetesClientMetrics (
90
+ MetricGroup parentGroup ,
91
+ FlinkOperatorConfiguration flinkOperatorConfiguration ,
92
+ LongSupplier nanoTimeSource ) {
93
+ this .nanoTimeSource = nanoTimeSource ;
75
94
MetricGroup metricGroup = parentGroup .addGroup (KUBE_CLIENT_GROUP );
76
95
77
96
this .requestMetricGroup = metricGroup .addGroup (HTTP_REQUEST_GROUP );
@@ -121,29 +140,86 @@ public KubernetesClientMetrics(
121
140
}
122
141
123
142
@ Override
124
- public Response intercept (Chain chain ) throws IOException {
125
- Request request = chain .request ();
143
+ public void before (BasicBuilder builder , HttpRequest request , RequestTags tags ) {
144
+ long requestStartTime = nanoTimeSource .getAsLong ();
145
+ // Attach a header to the request. We don't care if is actually sent or echoed back in the
146
+ // response.
147
+ // As the request is included in the after callbacks so we just read the value from the
148
+ // headers on that.
149
+ builder .setHeader (REQUEST_START_TIME_HEADER , String .valueOf (requestStartTime ));
126
150
updateRequestMetrics (request );
127
- Response response = null ;
128
- final long startTime = System .nanoTime ();
129
- try {
130
- response = chain .proceed (request );
131
- return response ;
132
- } finally {
133
- updateResponseMetrics (response , startTime );
134
- }
135
151
}
136
152
137
- private void updateRequestMetrics (Request request ) {
153
+ @ Override
154
+ public void after (
155
+ HttpRequest request ,
156
+ HttpResponse <?> response ,
157
+ AsyncBody .Consumer <List <ByteBuffer >> consumer ) {
158
+ trackRequestLatency (request );
159
+ updateResponseMetrics (response );
160
+ }
161
+
162
+ @ Override
163
+ public CompletableFuture <Boolean > afterFailure (
164
+ BasicBuilder builder , HttpResponse <?> response , RequestTags tags ) {
165
+ this .requestFailedRateMeter .markEvent ();
166
+ return CompletableFuture .completedFuture (false );
167
+ }
168
+
169
+ @ Override
170
+ public void afterConnectionFailure (HttpRequest request , Throwable failure ) {
171
+ trackRequestLatency (request );
172
+ this .requestFailedRateMeter .markEvent ();
173
+ }
174
+
175
+ @ VisibleForTesting
176
+ Counter getRequestCounter () {
177
+ return requestCounter ;
178
+ }
179
+
180
+ @ VisibleForTesting
181
+ Counter getResponseCounter () {
182
+ return responseCounter ;
183
+ }
184
+
185
+ @ VisibleForTesting
186
+ Counter getRequestMethodCounter (String method ) {
187
+ return requestMethodCounter .get (method );
188
+ }
189
+
190
+ @ VisibleForTesting
191
+ SynchronizedMeterView getRequestRateMeter () {
192
+ return requestRateMeter ;
193
+ }
194
+
195
+ @ VisibleForTesting
196
+ SynchronizedMeterView getResponseCodeMeter (int statusCode ) {
197
+ return responseCodeMeters .get (statusCode );
198
+ }
199
+
200
+ @ VisibleForTesting
201
+ List <SynchronizedMeterView > getResponseCodeGroupMeters () {
202
+ return responseCodeGroupMeters ;
203
+ }
204
+
205
+ @ VisibleForTesting
206
+ Histogram getResponseLatency () {
207
+ return responseLatency ;
208
+ }
209
+
210
+ @ VisibleForTesting
211
+ SynchronizedMeterView getRequestFailedRateMeter () {
212
+ return requestFailedRateMeter ;
213
+ }
214
+
215
+ private void updateRequestMetrics (HttpRequest request ) {
138
216
this .requestRateMeter .markEvent ();
139
217
getCounterByRequestMethod (request .method ()).inc ();
140
218
}
141
219
142
- private void updateResponseMetrics (Response response , long startTimeNanos ) {
143
- final long latency = System .nanoTime () - startTimeNanos ;
220
+ private void updateResponseMetrics (HttpResponse <?> response ) {
144
221
if (response != null ) {
145
222
this .responseRateMeter .markEvent ();
146
- this .responseLatency .update (latency );
147
223
getMeterViewByResponseCode (response .code ()).markEvent ();
148
224
if (this .httpResponseCodeGroupsEnabled ) {
149
225
responseCodeGroupMeters .get (response .code () / 100 - 1 ).markEvent ();
@@ -153,6 +229,16 @@ private void updateResponseMetrics(Response response, long startTimeNanos) {
153
229
}
154
230
}
155
231
232
+ private void trackRequestLatency (HttpRequest request ) {
233
+ final String header = request .header (REQUEST_START_TIME_HEADER );
234
+ if (header != null ) {
235
+ final long currentNanos = nanoTimeSource .getAsLong ();
236
+ final long requestStartNanos = Long .parseLong (header );
237
+ final long latency = currentNanos - requestStartNanos ;
238
+ this .responseLatency .update (latency );
239
+ }
240
+ }
241
+
156
242
private Counter getCounterByRequestMethod (String method ) {
157
243
return requestMethodCounter .computeIfAbsent (
158
244
method ,
0 commit comments