Hisoka-X commented on code in PR #9576:
URL: https://github.com/apache/seatunnel/pull/9576#discussion_r2382372274
##########
.github/workflows/backend.yml:
##########
@@ -874,7 +874,7 @@ jobs:
matrix:
java: [ '8', '11' ]
os: [ 'ubuntu-latest' ]
- timeout-minutes: 150
+ timeout-minutes: 210
Review Comment:
Could you adjust the timeout value base on real ci run time? We can not
increase it to too long.
##########
seatunnel-core/seatunnel-flink-starter/seatunnel-flink-20-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.seatunnel.core.starter.flink.execution;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import org.apache.seatunnel.translation.flink.metric.FlinkJobMetricsSummary;
+
+import org.apache.flink.api.common.JobExecutionResult;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Flink execution implementation for Flink 1.20. */
+public class FlinkExecution extends AbstractFlinkExecution {
+
+ private static final Logger LOGGER =
LoggerFactory.getLogger(FlinkExecution.class);
+
+ public FlinkExecution(Config config) {
+ super(config);
+ }
+
+ @Override
+ protected FlinkJobMetricsSummary createJobMetricsSummary(
+ JobExecutionResult jobResult, long jobStartTime, long jobEndTime) {
+ return createFlink20JobMetricsSummary(jobResult, jobStartTime,
jobEndTime);
+ }
+
+ protected FlinkJobMetricsSummary createFlink20JobMetricsSummary(
+ JobExecutionResult jobResult, long jobStartTime, long jobEndTime) {
+ String jobId = jobResult.getJobID().toString();
+
+ String metricPrefix = "seatunnel.metric." + jobId + ".";
+ System.getProperties().stringPropertyNames().stream()
+ .filter(name -> name.startsWith(metricPrefix))
+ .forEach(
+ name ->
+ LOGGER.info(
+ "System property metric: {} = {}",
+ name,
+ System.getProperty(name)));
Review Comment:
This part only print same log? why only flink 1.20 should do this?
##########
seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-20/src/main/java/org/apache/seatunnel/translation/flink/metric/FlinkMeter.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.seatunnel.translation.flink.metric;
+
+import org.apache.seatunnel.api.common.metrics.Meter;
+import org.apache.seatunnel.api.common.metrics.Unit;
+
+/** Flink implementation of SeaTunnel Meter metric. */
+public class FlinkMeter implements Meter {
+ private final org.apache.flink.metrics.Meter flinkMeter;
+
+ public FlinkMeter(org.apache.flink.metrics.Meter flinkMeter) {
+ this.flinkMeter = flinkMeter;
+ }
+
+ @Override
+ public void markEvent() {
+ flinkMeter.markEvent();
+ }
+
+ @Override
+ public void markEvent(long n) {
+ flinkMeter.markEvent(n);
+ }
+
+ @Override
+ public double getRate() {
+ return flinkMeter.getRate();
+ }
+
+ @Override
+ public long getCount() {
+ return 0;
+ }
+
+ @Override
+ public String name() {
+ return "";
Review Comment:
name?
##########
seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-20/src/main/java/org/apache/seatunnel/translation/flink/metric/FlinkCounter.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.seatunnel.translation.flink.metric;
+
+import org.apache.seatunnel.api.common.metrics.Counter;
+import org.apache.seatunnel.api.common.metrics.Unit;
+
+/** Flink implementation of SeaTunnel Counter metric. */
+public class FlinkCounter implements Counter {
+ private final org.apache.flink.metrics.Counter flinkCounter;
+
+ public FlinkCounter(org.apache.flink.metrics.Counter flinkCounter) {
+ this.flinkCounter = flinkCounter;
+ }
+
+ @Override
+ public void inc() {
+ flinkCounter.inc();
+ }
+
+ @Override
+ public void inc(long n) {
+ flinkCounter.inc(n);
+ }
+
+ @Override
+ public void dec() {
+ // Not supported by Flink counter
Review Comment:
please throw exception
##########
seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-20/src/main/java/org/apache/seatunnel/translation/flink/metric/FlinkJobMetricsSummary.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.seatunnel.translation.flink.metric;
+
+import org.apache.seatunnel.api.common.metrics.MetricNames;
+import org.apache.seatunnel.common.utils.DateTimeUtils;
+import org.apache.seatunnel.common.utils.StringFormatUtils;
+
+import org.apache.flink.api.common.JobExecutionResult;
+
+import lombok.extern.slf4j.Slf4j;
+
+import java.time.Duration;
+import java.time.LocalDateTime;
+import java.util.HashMap;
+import java.util.Map;
+
+@Slf4j
+public class FlinkJobMetricsSummary {
+
+ private final JobExecutionResult jobExecutionResult;
+ private final LocalDateTime jobStartTime;
+ private final LocalDateTime jobEndTime;
+
+ public FlinkJobMetricsSummary(
+ JobExecutionResult jobExecutionResult,
+ LocalDateTime jobStartTime,
+ LocalDateTime jobEndTime) {
+ this.jobExecutionResult = jobExecutionResult;
+ this.jobStartTime = jobStartTime;
+ this.jobEndTime = jobEndTime;
+ log.info(
+ "FlinkJobMetricsSummary created for job: {}",
+ jobExecutionResult != null ? jobExecutionResult.getJobID() :
"null");
+ }
+
+ public static Builder builder() {
+ return new Builder();
+ }
+
+ public static class Builder {
+ private JobExecutionResult jobExecutionResult;
+ private long jobStartTime;
+ private long jobEndTime;
+
+ private Builder() {}
+
+ public Builder jobExecutionResult(JobExecutionResult
jobExecutionResult) {
+ this.jobExecutionResult = jobExecutionResult;
+ return this;
+ }
+
+ public Builder jobStartTime(long jobStartTime) {
+ this.jobStartTime = jobStartTime;
+ return this;
+ }
+
+ public Builder jobEndTime(long jobEndTime) {
+ this.jobEndTime = jobEndTime;
+ return this;
+ }
+
+ public FlinkJobMetricsSummary build() {
+ return new FlinkJobMetricsSummary(
+ jobExecutionResult,
+ DateTimeUtils.parse(jobStartTime),
+ DateTimeUtils.parse(jobEndTime));
+ }
+ }
+
+ public Map<String, Object> getMetrics() {
+ Map<String, Object> metrics = new HashMap<>();
+
+ if (jobExecutionResult == null) {
+ log.warn("JobExecutionResult is null, cannot get metrics");
+ return metrics;
+ }
+
+ try {
+ Map<String, Object> accumulatorResults =
jobExecutionResult.getAllAccumulatorResults();
+
+ for (Map.Entry<String, Object> entry :
accumulatorResults.entrySet()) {
+ String key = entry.getKey();
+ Object value = entry.getValue();
+
+ if (value instanceof Number) {
+ long longValue = ((Number) value).longValue();
+
+ if (key.equals(MetricNames.SINK_WRITE_COUNT)
+ || key.contains("SinkWriteCount")) {
Review Comment:
when key.equals(MetricNames.SINK_WRITE_COUNT) is true. The
key.contains("SinkWriteCount") also is true.
##########
seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-20/src/main/java/org/apache/seatunnel/translation/flink/metric/FlinkAccumulatorCounter.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.seatunnel.translation.flink.metric;
+
+import org.apache.seatunnel.api.common.metrics.Counter;
+import org.apache.seatunnel.api.common.metrics.MetricNames;
+import org.apache.seatunnel.api.common.metrics.Unit;
+
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class FlinkAccumulatorCounter implements Counter {
+ private final String name;
+ private final org.apache.flink.metrics.Counter flinkCounter;
+ private final LongCounter accumulator;
+ private final RuntimeContext runtimeContext;
+ private volatile long localCount = 0L;
+
+ public FlinkAccumulatorCounter(
+ String name,
+ org.apache.flink.metrics.Counter flinkCounter,
+ RuntimeContext runtimeContext) {
+ this.name = name;
+ this.flinkCounter = flinkCounter;
+ this.runtimeContext = runtimeContext;
+ this.accumulator = new LongCounter();
+
+ try {
+ String accumulatorName = getStandardAccumulatorName(name);
+ runtimeContext.addAccumulator(accumulatorName, accumulator);
+ } catch (Exception e) {
+ log.warn("Failed to register accumulator: {}", name);
+ }
+ }
+
+ @Override
+ public void inc() {
+ inc(1L);
+ }
+
+ @Override
+ public void inc(long n) {
+ try {
+ if (flinkCounter != null) {
+ flinkCounter.inc(n);
+ }
+
+ accumulator.add(n);
+
+ localCount += n;
+
+ } catch (Exception e) {
+ log.warn("Error incrementing counter {}", name);
+ localCount += n;
+ }
Review Comment:
why catch Exception? We should direct throw it.
##########
seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-20/src/main/java/org/apache/seatunnel/translation/flink/metric/FlinkMetricContext.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.seatunnel.translation.flink.metric;
+
+import org.apache.seatunnel.api.common.metrics.Counter;
+import org.apache.seatunnel.api.common.metrics.Meter;
+import org.apache.seatunnel.api.common.metrics.MetricNames;
+import org.apache.seatunnel.api.common.metrics.MetricsContext;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+
+import lombok.extern.slf4j.Slf4j;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+@Slf4j
+public class FlinkMetricContext implements MetricsContext {
+
+ private final MetricGroup metricGroup;
+ private final StreamingRuntimeContext runtimeContext;
+ private final RuntimeContext generalRuntimeContext;
+ private final Map<String, Counter> counters = new ConcurrentHashMap<>();
+ private final Map<String, Meter> meters = new ConcurrentHashMap<>();
+
+ public FlinkMetricContext(StreamingRuntimeContext runtimeContext) {
+ this.runtimeContext = runtimeContext;
+ this.generalRuntimeContext = runtimeContext;
+ this.metricGroup = runtimeContext != null ?
runtimeContext.getMetricGroup() : null;
+ }
+
+ public FlinkMetricContext(RuntimeContext runtimeContext, MetricGroup
metricGroup) {
+ this.runtimeContext =
+ runtimeContext instanceof StreamingRuntimeContext
+ ? (StreamingRuntimeContext) runtimeContext
+ : null;
+ this.generalRuntimeContext = runtimeContext;
+ this.metricGroup = metricGroup;
+ }
+
+ public FlinkMetricContext(MetricGroup metricGroup) {
+ this.metricGroup = metricGroup;
+ this.generalRuntimeContext = null;
+ this.runtimeContext = null;
+ }
+
+ @Override
+ public Counter counter(String name) {
+ Counter existingCounter = counters.get(name);
+ if (existingCounter != null) {
+ return existingCounter;
+ }
+
+ if (metricGroup == null) {
+ Counter noOpCounter = new NoOpCounter();
Review Comment:
why we need NoOpCounter?
##########
seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-20/src/main/java/org/apache/seatunnel/translation/flink/metric/FlinkAccumulatorCounter.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.seatunnel.translation.flink.metric;
+
+import org.apache.seatunnel.api.common.metrics.Counter;
+import org.apache.seatunnel.api.common.metrics.MetricNames;
+import org.apache.seatunnel.api.common.metrics.Unit;
+
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class FlinkAccumulatorCounter implements Counter {
+ private final String name;
+ private final org.apache.flink.metrics.Counter flinkCounter;
+ private final LongCounter accumulator;
+ private final RuntimeContext runtimeContext;
+ private volatile long localCount = 0L;
+
+ public FlinkAccumulatorCounter(
+ String name,
+ org.apache.flink.metrics.Counter flinkCounter,
+ RuntimeContext runtimeContext) {
+ this.name = name;
+ this.flinkCounter = flinkCounter;
+ this.runtimeContext = runtimeContext;
+ this.accumulator = new LongCounter();
+
+ try {
+ String accumulatorName = getStandardAccumulatorName(name);
+ runtimeContext.addAccumulator(accumulatorName, accumulator);
+ } catch (Exception e) {
+ log.warn("Failed to register accumulator: {}", name);
+ }
+ }
+
+ @Override
+ public void inc() {
+ inc(1L);
+ }
+
+ @Override
+ public void inc(long n) {
+ try {
+ if (flinkCounter != null) {
+ flinkCounter.inc(n);
+ }
+
+ accumulator.add(n);
+
+ localCount += n;
Review Comment:
it's not thread safe. Please use AtomicLong instead.
##########
seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-20/src/main/java/org/apache/seatunnel/translation/flink/metric/FlinkJobMetricsSummary.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.seatunnel.translation.flink.metric;
+
+import org.apache.seatunnel.api.common.metrics.MetricNames;
+import org.apache.seatunnel.common.utils.DateTimeUtils;
+import org.apache.seatunnel.common.utils.StringFormatUtils;
+
+import org.apache.flink.api.common.JobExecutionResult;
+
+import lombok.extern.slf4j.Slf4j;
+
+import java.time.Duration;
+import java.time.LocalDateTime;
+import java.util.HashMap;
+import java.util.Map;
+
+@Slf4j
+public class FlinkJobMetricsSummary {
+
+ private final JobExecutionResult jobExecutionResult;
+ private final LocalDateTime jobStartTime;
+ private final LocalDateTime jobEndTime;
+
+ public FlinkJobMetricsSummary(
+ JobExecutionResult jobExecutionResult,
+ LocalDateTime jobStartTime,
+ LocalDateTime jobEndTime) {
+ this.jobExecutionResult = jobExecutionResult;
+ this.jobStartTime = jobStartTime;
+ this.jobEndTime = jobEndTime;
+ log.info(
+ "FlinkJobMetricsSummary created for job: {}",
+ jobExecutionResult != null ? jobExecutionResult.getJobID() :
"null");
+ }
+
+ public static Builder builder() {
+ return new Builder();
+ }
+
+ public static class Builder {
+ private JobExecutionResult jobExecutionResult;
+ private long jobStartTime;
+ private long jobEndTime;
+
+ private Builder() {}
+
+ public Builder jobExecutionResult(JobExecutionResult
jobExecutionResult) {
+ this.jobExecutionResult = jobExecutionResult;
+ return this;
+ }
+
+ public Builder jobStartTime(long jobStartTime) {
+ this.jobStartTime = jobStartTime;
+ return this;
+ }
+
+ public Builder jobEndTime(long jobEndTime) {
+ this.jobEndTime = jobEndTime;
+ return this;
+ }
+
+ public FlinkJobMetricsSummary build() {
+ return new FlinkJobMetricsSummary(
+ jobExecutionResult,
+ DateTimeUtils.parse(jobStartTime),
+ DateTimeUtils.parse(jobEndTime));
+ }
+ }
+
+ public Map<String, Object> getMetrics() {
+ Map<String, Object> metrics = new HashMap<>();
+
+ if (jobExecutionResult == null) {
+ log.warn("JobExecutionResult is null, cannot get metrics");
+ return metrics;
+ }
+
+ try {
+ Map<String, Object> accumulatorResults =
jobExecutionResult.getAllAccumulatorResults();
+
+ for (Map.Entry<String, Object> entry :
accumulatorResults.entrySet()) {
+ String key = entry.getKey();
+ Object value = entry.getValue();
+
+ if (value instanceof Number) {
+ long longValue = ((Number) value).longValue();
+
+ if (key.equals(MetricNames.SINK_WRITE_COUNT)
+ || key.contains("SinkWriteCount")) {
Review Comment:
```suggestion
if (key.contains("SinkWriteCount")) {
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]