This is an automated email from the ASF dual-hosted git repository.

wanghailin pushed a commit to branch dev
in repository https://gitbox.apache.org/repos/asf/seatunnel.git


The following commit(s) were added to refs/heads/dev by this push:
     new 0e67d9e049 [Zeta][Log] Fix trace parallel stream (#8206)
0e67d9e049 is described below

commit 0e67d9e04926dfaf957f34d3af3668b378bda313
Author: hailin0 <wanghai...@apache.org>
AuthorDate: Thu Dec 5 19:55:45 2024 +0800

    [Zeta][Log] Fix trace parallel stream (#8206)
---
 .../seatunnel/api/tracing/MDCComparator.java       |  44 ++++
 .../apache/seatunnel/api/tracing/MDCConsumer.java  |  44 ++++
 .../apache/seatunnel/api/tracing/MDCFunction.java  |  44 ++++
 .../apache/seatunnel/api/tracing/MDCPredicate.java |  44 ++++
 .../apache/seatunnel/api/tracing/MDCStream.java    | 255 +++++++++++++++++++++
 .../apache/seatunnel/api/tracing/MDCTracer.java    |  64 ++++++
 .../seatunnel/api/tracing/MDCTracerTest.java       |  79 +++++++
 .../server/checkpoint/CheckpointManager.java       |   9 +-
 .../engine/server/task/SeaTunnelTask.java          |   4 +-
 9 files changed, 579 insertions(+), 8 deletions(-)

diff --git 
a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCComparator.java
 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCComparator.java
new file mode 100644
index 0000000000..5672cf4392
--- /dev/null
+++ 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCComparator.java
@@ -0,0 +1,44 @@
+/*
+ * 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.api.tracing;
+
+import java.util.Comparator;
+
+public class MDCComparator<T> implements Comparator<T> {
+    private final MDCContext context;
+    private final Comparator<T> delegate;
+
+    public MDCComparator(Comparator<T> delegate) {
+        this(MDCContext.current(), delegate);
+    }
+
+    public MDCComparator(MDCContext context, Comparator<T> delegate) {
+        this.context = context;
+        this.delegate = delegate;
+    }
+
+    @Override
+    public int compare(T o1, T o2) {
+        try {
+            context.put();
+            return delegate.compare(o1, o2);
+        } finally {
+            context.clear();
+        }
+    }
+}
diff --git 
a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCConsumer.java 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCConsumer.java
new file mode 100644
index 0000000000..6317f038a0
--- /dev/null
+++ 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCConsumer.java
@@ -0,0 +1,44 @@
+/*
+ * 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.api.tracing;
+
+import java.util.function.Consumer;
+
+public class MDCConsumer<T> implements Consumer<T> {
+    private final MDCContext context;
+    private final Consumer<T> delegate;
+
+    public MDCConsumer(Consumer<T> delegate) {
+        this(MDCContext.current(), delegate);
+    }
+
+    public MDCConsumer(MDCContext context, Consumer<T> delegate) {
+        this.context = context;
+        this.delegate = delegate;
+    }
+
+    @Override
+    public void accept(T t) {
+        try {
+            context.put();
+            delegate.accept(t);
+        } finally {
+            context.clear();
+        }
+    }
+}
diff --git 
a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCFunction.java 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCFunction.java
new file mode 100644
index 0000000000..b8f0d22248
--- /dev/null
+++ 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCFunction.java
@@ -0,0 +1,44 @@
+/*
+ * 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.api.tracing;
+
+import java.util.function.Function;
+
+public class MDCFunction<T, R> implements Function<T, R> {
+    private final MDCContext context;
+    protected final Function<T, R> delegate;
+
+    public MDCFunction(Function<T, R> delegate) {
+        this(MDCContext.current(), delegate);
+    }
+
+    public MDCFunction(MDCContext context, Function<T, R> delegate) {
+        this.context = context;
+        this.delegate = delegate;
+    }
+
+    @Override
+    public R apply(T t) {
+        try {
+            context.put();
+            return delegate.apply(t);
+        } finally {
+            context.clear();
+        }
+    }
+}
diff --git 
a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCPredicate.java
 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCPredicate.java
new file mode 100644
index 0000000000..d04edc064b
--- /dev/null
+++ 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCPredicate.java
@@ -0,0 +1,44 @@
+/*
+ * 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.api.tracing;
+
+import java.util.function.Predicate;
+
+public class MDCPredicate<T> implements Predicate<T> {
+    private final MDCContext context;
+    private final Predicate<T> delegate;
+
+    public MDCPredicate(Predicate<T> delegate) {
+        this(MDCContext.current(), delegate);
+    }
+
+    public MDCPredicate(MDCContext context, Predicate<T> delegate) {
+        this.context = context;
+        this.delegate = delegate;
+    }
+
+    @Override
+    public boolean test(T t) {
+        try {
+            context.put();
+            return delegate.test(t);
+        } finally {
+            context.clear();
+        }
+    }
+}
diff --git 
a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCStream.java 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCStream.java
new file mode 100644
index 0000000000..871af5cd60
--- /dev/null
+++ 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCStream.java
@@ -0,0 +1,255 @@
+/*
+ * 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.api.tracing;
+
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Optional;
+import java.util.Spliterator;
+import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
+import java.util.function.BinaryOperator;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.IntFunction;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+import java.util.function.ToDoubleFunction;
+import java.util.function.ToIntFunction;
+import java.util.function.ToLongFunction;
+import java.util.stream.Collector;
+import java.util.stream.DoubleStream;
+import java.util.stream.IntStream;
+import java.util.stream.LongStream;
+import java.util.stream.Stream;
+
+public class MDCStream<T> implements Stream<T> {
+    private final MDCContext context;
+    private final Stream<T> delegate;
+
+    public MDCStream(Stream<T> delegate) {
+        this(MDCContext.current(), delegate);
+    }
+
+    public MDCStream(MDCContext context, Stream<T> delegate) {
+        this.context = context;
+        this.delegate = delegate;
+    }
+
+    @Override
+    public Stream<T> filter(Predicate<? super T> predicate) {
+        return new MDCStream<>(context, delegate.filter(new 
MDCPredicate<>(context, predicate)));
+    }
+
+    @Override
+    public <R> Stream<R> map(Function<? super T, ? extends R> mapper) {
+        return new MDCStream<>(context, delegate.map(new 
MDCFunction<>(context, mapper)));
+    }
+
+    @Override
+    public <R> Stream<R> flatMap(Function<? super T, ? extends Stream<? 
extends R>> mapper) {
+        return new MDCStream<>(context, delegate.flatMap(new 
MDCFunction<>(context, mapper)));
+    }
+
+    @Override
+    public Stream<T> sorted(Comparator<? super T> comparator) {
+        return new MDCStream<>(context, delegate.sorted(new 
MDCComparator<>(context, comparator)));
+    }
+
+    @Override
+    public Stream<T> peek(Consumer<? super T> action) {
+        return new MDCStream<>(context, delegate.peek(new 
MDCConsumer<>(context, action)));
+    }
+
+    @Override
+    public void forEach(Consumer<? super T> action) {
+        delegate.forEach(new MDCConsumer<>(context, action));
+    }
+
+    @Override
+    public void forEachOrdered(Consumer<? super T> action) {
+        delegate.forEachOrdered(new MDCConsumer<>(context, action));
+    }
+
+    @Override
+    public Optional<T> min(Comparator<? super T> comparator) {
+        return delegate.min(new MDCComparator<>(context, comparator));
+    }
+
+    @Override
+    public Optional<T> max(Comparator<? super T> comparator) {
+        return delegate.max(new MDCComparator<>(context, comparator));
+    }
+
+    @Override
+    public boolean anyMatch(Predicate<? super T> predicate) {
+        return delegate.anyMatch(new MDCPredicate<>(context, predicate));
+    }
+
+    @Override
+    public boolean allMatch(Predicate<? super T> predicate) {
+        return delegate.allMatch(new MDCPredicate<>(context, predicate));
+    }
+
+    @Override
+    public boolean noneMatch(Predicate<? super T> predicate) {
+        return delegate.noneMatch(new MDCPredicate<>(context, predicate));
+    }
+
+    @Override
+    public Stream<T> onClose(Runnable closeHandler) {
+        return delegate.onClose(new MDCRunnable(context, closeHandler));
+    }
+
+    @Override
+    public Stream<T> sequential() {
+        return new MDCStream<>(context, delegate.sequential());
+    }
+
+    @Override
+    public Stream<T> parallel() {
+        return new MDCStream<>(context, delegate.parallel());
+    }
+
+    @Override
+    public Stream<T> unordered() {
+        return new MDCStream<>(context, delegate.unordered());
+    }
+
+    @Override
+    public Stream<T> distinct() {
+        return new MDCStream<>(context, delegate.distinct());
+    }
+
+    @Override
+    public Stream<T> sorted() {
+        return new MDCStream<>(context, delegate.sorted());
+    }
+
+    @Override
+    public Stream<T> limit(long maxSize) {
+        return new MDCStream<>(context, delegate.limit(maxSize));
+    }
+
+    @Override
+    public Stream<T> skip(long n) {
+        return new MDCStream<>(context, delegate.skip(n));
+    }
+
+    @Override
+    public IntStream flatMapToInt(Function<? super T, ? extends IntStream> 
mapper) {
+        return delegate.flatMapToInt(new MDCFunction<>(context, mapper));
+    }
+
+    @Override
+    public LongStream flatMapToLong(Function<? super T, ? extends LongStream> 
mapper) {
+        return delegate.flatMapToLong(new MDCFunction<>(context, mapper));
+    }
+
+    @Override
+    public DoubleStream flatMapToDouble(Function<? super T, ? extends 
DoubleStream> mapper) {
+        return delegate.flatMapToDouble(new MDCFunction<>(context, mapper));
+    }
+
+    @Override
+    public IntStream mapToInt(ToIntFunction<? super T> mapper) {
+        return delegate.mapToInt(mapper);
+    }
+
+    @Override
+    public LongStream mapToLong(ToLongFunction<? super T> mapper) {
+        return delegate.mapToLong(mapper);
+    }
+
+    @Override
+    public DoubleStream mapToDouble(ToDoubleFunction<? super T> mapper) {
+        return delegate.mapToDouble(mapper);
+    }
+
+    @Override
+    public Object[] toArray() {
+        return delegate.toArray();
+    }
+
+    @Override
+    public <A> A[] toArray(IntFunction<A[]> generator) {
+        return delegate.toArray(generator);
+    }
+
+    @Override
+    public T reduce(T identity, BinaryOperator<T> accumulator) {
+        return delegate.reduce(identity, accumulator);
+    }
+
+    @Override
+    public Optional<T> reduce(BinaryOperator<T> accumulator) {
+        return delegate.reduce(accumulator);
+    }
+
+    @Override
+    public <U> U reduce(
+            U identity, BiFunction<U, ? super T, U> accumulator, 
BinaryOperator<U> combiner) {
+        return delegate.reduce(identity, accumulator, combiner);
+    }
+
+    @Override
+    public <R> R collect(
+            Supplier<R> supplier, BiConsumer<R, ? super T> accumulator, 
BiConsumer<R, R> combiner) {
+        return delegate.collect(supplier, accumulator, combiner);
+    }
+
+    @Override
+    public <R, A> R collect(Collector<? super T, A, R> collector) {
+        return delegate.collect(collector);
+    }
+
+    @Override
+    public long count() {
+        return delegate.count();
+    }
+
+    @Override
+    public Optional<T> findFirst() {
+        return delegate.findFirst();
+    }
+
+    @Override
+    public Optional<T> findAny() {
+        return delegate.findAny();
+    }
+
+    @Override
+    public Iterator<T> iterator() {
+        return delegate.iterator();
+    }
+
+    @Override
+    public Spliterator<T> spliterator() {
+        return delegate.spliterator();
+    }
+
+    @Override
+    public boolean isParallel() {
+        return delegate.isParallel();
+    }
+
+    @Override
+    public void close() {
+        delegate.close();
+    }
+}
diff --git 
a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCTracer.java 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCTracer.java
index 1ce2120cc3..d42242aeaf 100644
--- 
a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCTracer.java
+++ 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCTracer.java
@@ -21,6 +21,10 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ScheduledExecutorService;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
 
 /**
  * Tracer for MDC context.
@@ -123,4 +127,64 @@ public class MDCTracer {
         }
         return new MDCScheduledExecutorService(context, delegate);
     }
+
+    public static <T> MDCConsumer<T> tracing(Consumer<T> delegate) {
+        return tracing(MDCContext.current(), delegate);
+    }
+
+    public static <T> MDCConsumer<T> tracing(Long jobId, Consumer<T> delegate) 
{
+        return tracing(MDCContext.of(jobId), delegate);
+    }
+
+    public static <T> MDCConsumer<T> tracing(MDCContext context, Consumer<T> 
delegate) {
+        if (delegate instanceof MDCConsumer) {
+            throw new IllegalArgumentException("Already an MDCConsumer");
+        }
+        return new MDCConsumer<>(context, delegate);
+    }
+
+    public static <T, R> MDCFunction<T, R> tracing(Function<T, R> delegate) {
+        return tracing(MDCContext.current(), delegate);
+    }
+
+    public static <T, R> MDCFunction<T, R> tracing(Long jobId, Function<T, R> 
delegate) {
+        return tracing(MDCContext.of(jobId), delegate);
+    }
+
+    public static <T, R> MDCFunction<T, R> tracing(MDCContext context, 
Function<T, R> delegate) {
+        if (delegate instanceof MDCFunction) {
+            throw new IllegalArgumentException("Already an MDCFunction");
+        }
+        return new MDCFunction<>(context, delegate);
+    }
+
+    public static <T> MDCPredicate<T> tracing(Predicate<T> delegate) {
+        return tracing(MDCContext.current(), delegate);
+    }
+
+    public static <T> MDCPredicate<T> tracing(Long jobId, Predicate<T> 
delegate) {
+        return tracing(MDCContext.of(jobId), delegate);
+    }
+
+    public static <T> MDCPredicate<T> tracing(MDCContext context, Predicate<T> 
delegate) {
+        if (delegate instanceof MDCPredicate) {
+            throw new IllegalArgumentException("Already an MDCPredicate");
+        }
+        return new MDCPredicate<>(context, delegate);
+    }
+
+    public static <T> MDCStream<T> tracing(Stream<T> delegate) {
+        return tracing(MDCContext.current(), delegate);
+    }
+
+    public static <T> MDCStream<T> tracing(Long jobId, Stream<T> delegate) {
+        return tracing(MDCContext.of(jobId), delegate);
+    }
+
+    public static <T> MDCStream<T> tracing(MDCContext context, Stream<T> 
delegate) {
+        if (delegate instanceof MDCStream) {
+            throw new IllegalArgumentException("Already an MDCStream");
+        }
+        return new MDCStream<>(context, delegate);
+    }
 }
diff --git 
a/seatunnel-api/src/test/java/org/apache/seatunnel/api/tracing/MDCTracerTest.java
 
b/seatunnel-api/src/test/java/org/apache/seatunnel/api/tracing/MDCTracerTest.java
index 88ac95885f..9d8049f1fb 100644
--- 
a/seatunnel-api/src/test/java/org/apache/seatunnel/api/tracing/MDCTracerTest.java
+++ 
b/seatunnel-api/src/test/java/org/apache/seatunnel/api/tracing/MDCTracerTest.java
@@ -21,8 +21,10 @@ import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 import org.slf4j.MDC;
 
+import java.util.Arrays;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Executors;
+import java.util.stream.Stream;
 
 public class MDCTracerTest {
 
@@ -125,6 +127,83 @@ public class MDCTracerTest {
         Assertions.assertNull(MDC.get(MDCContext.TASK_ID));
     }
 
+    @Test
+    public void testMDCTracedStream() throws Exception {
+        MDCContext mdcContext = MDCContext.of(1, 2, 3);
+
+        Stream<Integer> tracedStream =
+                MDCTracer.tracing(mdcContext, Arrays.asList(1, 2, 
3).parallelStream());
+
+        Assertions.assertNull(MDC.get(MDCContext.JOB_ID));
+        Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID));
+        Assertions.assertNull(MDC.get(MDCContext.TASK_ID));
+        tracedStream
+                .filter(
+                        integer -> {
+                            Assertions.assertEquals("1", 
MDC.get(MDCContext.JOB_ID));
+                            Assertions.assertEquals("2", 
MDC.get(MDCContext.PIPELINE_ID));
+                            Assertions.assertEquals("3", 
MDC.get(MDCContext.TASK_ID));
+                            return true;
+                        })
+                .map(
+                        integer -> {
+                            Assertions.assertEquals("1", 
MDC.get(MDCContext.JOB_ID));
+                            Assertions.assertEquals("2", 
MDC.get(MDCContext.PIPELINE_ID));
+                            Assertions.assertEquals("3", 
MDC.get(MDCContext.TASK_ID));
+                            return integer;
+                        })
+                .sorted(
+                        (o1, o2) -> {
+                            Assertions.assertEquals("1", 
MDC.get(MDCContext.JOB_ID));
+                            Assertions.assertEquals("2", 
MDC.get(MDCContext.PIPELINE_ID));
+                            Assertions.assertEquals("3", 
MDC.get(MDCContext.TASK_ID));
+                            return Integer.compare(o1, o2);
+                        })
+                .forEach(
+                        integer -> {
+                            Assertions.assertEquals("1", 
MDC.get(MDCContext.JOB_ID));
+                            Assertions.assertEquals("2", 
MDC.get(MDCContext.PIPELINE_ID));
+                            Assertions.assertEquals("3", 
MDC.get(MDCContext.TASK_ID));
+                        });
+        Assertions.assertNull(MDC.get(MDCContext.JOB_ID));
+        Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID));
+        Assertions.assertNull(MDC.get(MDCContext.TASK_ID));
+
+        tracedStream = MDCTracer.tracing(mdcContext, Arrays.asList(1, 2, 
3).stream());
+
+        Assertions.assertNull(MDC.get(MDCContext.JOB_ID));
+        Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID));
+        Assertions.assertNull(MDC.get(MDCContext.TASK_ID));
+        tracedStream
+                .filter(
+                        integer -> {
+                            Assertions.assertEquals("1", 
MDC.get(MDCContext.JOB_ID));
+                            Assertions.assertEquals("2", 
MDC.get(MDCContext.PIPELINE_ID));
+                            Assertions.assertEquals("3", 
MDC.get(MDCContext.TASK_ID));
+                            return true;
+                        })
+                .map(
+                        integer -> {
+                            Assertions.assertEquals("1", 
MDC.get(MDCContext.JOB_ID));
+                            Assertions.assertEquals("2", 
MDC.get(MDCContext.PIPELINE_ID));
+                            Assertions.assertEquals("3", 
MDC.get(MDCContext.TASK_ID));
+                            return integer;
+                        })
+                .sorted(
+                        (o1, o2) -> {
+                            Assertions.assertEquals("1", 
MDC.get(MDCContext.JOB_ID));
+                            Assertions.assertEquals("2", 
MDC.get(MDCContext.PIPELINE_ID));
+                            Assertions.assertEquals("3", 
MDC.get(MDCContext.TASK_ID));
+                            return Integer.compare(o1, o2);
+                        })
+                .forEach(
+                        integer -> {
+                            Assertions.assertEquals("1", 
MDC.get(MDCContext.JOB_ID));
+                            Assertions.assertEquals("2", 
MDC.get(MDCContext.PIPELINE_ID));
+                            Assertions.assertEquals("3", 
MDC.get(MDCContext.TASK_ID));
+                        });
+    }
+
     @Test
     public void testMDCContext() throws Exception {
         MDCContext.current();
diff --git 
a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointManager.java
 
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointManager.java
index 122d263c34..0bede51c70 100644
--- 
a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointManager.java
+++ 
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointManager.java
@@ -19,6 +19,7 @@ package org.apache.seatunnel.engine.server.checkpoint;
 
 import 
org.apache.seatunnel.shade.com.google.common.annotations.VisibleForTesting;
 
+import org.apache.seatunnel.api.tracing.MDCTracer;
 import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
 import org.apache.seatunnel.engine.checkpoint.storage.api.CheckpointStorage;
 import 
org.apache.seatunnel.engine.checkpoint.storage.api.CheckpointStorageFactory;
@@ -105,9 +106,7 @@ public class CheckpointManager {
                                 checkpointConfig.getStorage().getStorage())
                         
.create(checkpointConfig.getStorage().getStoragePluginConfig());
         this.coordinatorMap =
-                checkpointPlanMap
-                        .values()
-                        .parallelStream()
+                MDCTracer.tracing(checkpointPlanMap.values().parallelStream())
                         .map(
                                 plan -> {
                                     IMapCheckpointIDCounter idCounter =
@@ -159,9 +158,7 @@ public class CheckpointManager {
      */
     @SuppressWarnings("unchecked")
     public PassiveCompletableFuture<CompletedCheckpoint>[] triggerSavePoints() 
{
-        return coordinatorMap
-                .values()
-                .parallelStream()
+        return MDCTracer.tracing(coordinatorMap.values().parallelStream())
                 .map(CheckpointCoordinator::startSavepoint)
                 .toArray(PassiveCompletableFuture[]::new);
     }
diff --git 
a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java
 
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java
index d66921c742..90b05d8704 100644
--- 
a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java
+++ 
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java
@@ -21,6 +21,7 @@ import org.apache.seatunnel.api.common.metrics.MetricTags;
 import org.apache.seatunnel.api.common.metrics.MetricsContext;
 import org.apache.seatunnel.api.table.type.Record;
 import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.api.tracing.MDCTracer;
 import org.apache.seatunnel.common.utils.function.ConsumerWithException;
 import org.apache.seatunnel.engine.core.checkpoint.InternalCheckpointListener;
 import org.apache.seatunnel.engine.core.dag.actions.Action;
@@ -322,8 +323,7 @@ public abstract class SeaTunnelTask extends AbstractTask {
     @Override
     public void close() throws IOException {
         super.close();
-        allCycles
-                .parallelStream()
+        MDCTracer.tracing(allCycles.parallelStream())
                 .forEach(
                         flowLifeCycle -> {
                             try {

Reply via email to