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

fanjia pushed a commit to branch st-engine
in repository https://gitbox.apache.org/repos/asf/incubator-seatunnel.git


The following commit(s) were added to refs/heads/st-engine by this push:
     new ab471411d add Task and TaskExecutionService Basic implementation 
(#2287)
ab471411d is described below

commit ab471411d46818a11613a0a9ebc6d7bc4081bfd5
Author: ic4y <[email protected]>
AuthorDate: Mon Aug 1 11:16:09 2022 +0800

    add Task and TaskExecutionService Basic implementation (#2287)
---
 .../seatunnel/engine/server/SeaTunnelServer.java   |   9 ++
 .../engine/server/TaskExecutionService.java        | 163 +++++++++++++++++++++
 .../engine/server/execution/ProgressState.java     |  48 ++++++
 .../seatunnel/engine/server/execution/Task.java    |  46 ++++++
 .../server/execution/TaskExecutionContext.java     |  49 +++++++
 .../engine/server/execution/TaskGroup.java         |  30 ++++
 .../test/execution/TaskExecutionServiceTest.java   |  91 ++++++++++++
 .../src/main/test/execution/TestTask.java          |  63 ++++++++
 8 files changed, 499 insertions(+)

diff --git 
a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java
 
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java
index 002b2558c..51a36f373 100644
--- 
a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java
+++ 
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java
@@ -35,14 +35,23 @@ public class SeaTunnelServer implements ManagedService, 
MembershipAwareService,
     private NodeEngineImpl nodeEngine;
     private final ILogger logger;
 
+    private TaskExecutionService taskExecutionService;
+
     public SeaTunnelServer(Node node) {
         this.logger = node.getLogger(getClass());
         logger.info("SeaTunnel server start...");
     }
 
+    public TaskExecutionService getTaskExecutionService(){
+        return this.taskExecutionService;
+    }
+
     @Override
     public void init(NodeEngine engine, Properties hzProperties) {
         this.nodeEngine = (NodeEngineImpl) engine;
+        taskExecutionService = new TaskExecutionService(
+            nodeEngine, nodeEngine.getProperties()
+        );
     }
 
     @Override
diff --git 
a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java
 
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java
new file mode 100644
index 000000000..fb5eed305
--- /dev/null
+++ 
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java
@@ -0,0 +1,163 @@
+/*
+ * 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.engine.server;
+
+import static com.hazelcast.jet.impl.util.ExceptionUtil.withTryCatch;
+import static java.util.concurrent.Executors.newCachedThreadPool;
+
+import org.apache.seatunnel.engine.server.execution.ProgressState;
+import org.apache.seatunnel.engine.server.execution.Task;
+import org.apache.seatunnel.engine.server.execution.TaskExecutionContext;
+import org.apache.seatunnel.engine.server.execution.TaskGroup;
+
+import com.hazelcast.jet.impl.util.NonCompletableFuture;
+import com.hazelcast.logging.ILogger;
+import com.hazelcast.spi.impl.NodeEngine;
+import com.hazelcast.spi.impl.NodeEngineImpl;
+import com.hazelcast.spi.properties.HazelcastProperties;
+import lombok.NonNull;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * This class is responsible for the execution of the Task
+ */
+public class TaskExecutionService {
+
+    private final String hzInstanceName;
+    private final NodeEngine nodeEngine;
+    private final ILogger logger;
+    private volatile boolean isShutdown;
+    private final ExecutorService blockingTaskletExecutor = 
newCachedThreadPool(new BlockingTaskThreadFactory());
+    // key: TaskID
+    private final ConcurrentMap<Long, TaskExecutionContext> executionContexts 
= new ConcurrentHashMap<>();
+
+    public TaskExecutionService(NodeEngineImpl nodeEngine, HazelcastProperties 
properties) {
+        this.hzInstanceName = nodeEngine.getHazelcastInstance().getName();
+        this.nodeEngine = nodeEngine;
+        this.logger = 
nodeEngine.getLoggingService().getLogger(TaskExecutionService.class);
+    }
+
+    public void shutdown() {
+        isShutdown = true;
+        blockingTaskletExecutor.shutdownNow();
+    }
+
+    public TaskExecutionContext getExecutionContext(long taskId) {
+        return executionContexts.get(taskId);
+    }
+
+    /**
+     * Submit a TaskGroup and run the Task in it
+     */
+    public Map<Long, TaskExecutionContext> submitTask(
+        TaskGroup taskGroup
+    ) {
+        Map<Long, TaskExecutionContext> contextMap = new 
HashMap<>(taskGroup.getTasks().size());
+        taskGroup.getTasks().forEach(task -> {
+            contextMap.put(task.getTaskID(), submitTask(task));
+        });
+        return contextMap;
+    }
+
+    public TaskExecutionContext submitTask(Task task) {
+        CompletableFuture<Void> cancellationFuture = new 
CompletableFuture<Void>();
+        TaskletTracker taskletTracker = new TaskletTracker(task, 
cancellationFuture);
+        taskletTracker.taskletFutures =
+            blockingTaskletExecutor.submit(new BlockingWorker(taskletTracker));
+
+        TaskExecutionContext taskExecutionContext = new TaskExecutionContext(
+            taskletTracker.future,
+            cancellationFuture,
+            this
+        );
+
+        executionContexts.put(task.getTaskID(), taskExecutionContext);
+        return taskExecutionContext;
+
+    }
+
+    private final class TaskletTracker {
+        final NonCompletableFuture future = new NonCompletableFuture();
+        final Task task;
+        volatile Future<?> taskletFutures;
+
+        TaskletTracker(Task task, CompletableFuture<Void> cancellationFuture) {
+            this.task = task;
+
+            cancellationFuture.whenComplete(withTryCatch(logger, (r, e) -> {
+                if (e == null) {
+                    e = new IllegalStateException("cancellationFuture should 
be completed exceptionally");
+                }
+                future.internalCompleteExceptionally(e);
+                taskletFutures.cancel(true);
+            }));
+        }
+
+        @Override
+        public String toString() {
+            return "Tracking " + task;
+        }
+    }
+
+    private final class BlockingWorker implements Runnable {
+
+        private final TaskletTracker tracker;
+
+        private BlockingWorker(TaskletTracker tracker) {
+            this.tracker = tracker;
+        }
+
+        @Override
+        public void run() {
+            final Task t = tracker.task;
+            try {
+                t.init();
+                ProgressState result;
+                do {
+                    result = t.call();
+                } while (!result.isDone() && !isShutdown && 
!tracker.taskletFutures.isCancelled());
+
+            } catch (Throwable e) {
+                logger.warning("Exception in " + t, e);
+                tracker.future.internalCompleteExceptionally(e);
+            } finally {
+                tracker.future.internalComplete();
+            }
+        }
+    }
+
+    private final class BlockingTaskThreadFactory implements ThreadFactory {
+        private final AtomicInteger seq = new AtomicInteger();
+
+        @Override
+        public Thread newThread(@NonNull Runnable r) {
+            return new Thread(r,
+                String.format("hz.%s.seaTunnel.blocking.thread-%d", 
hzInstanceName, seq.getAndIncrement()));
+        }
+    }
+
+}
diff --git 
a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/ProgressState.java
 
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/ProgressState.java
new file mode 100644
index 000000000..c5ec09ecf
--- /dev/null
+++ 
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/ProgressState.java
@@ -0,0 +1,48 @@
+/*
+ * 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.engine.server.execution;
+
+public enum ProgressState {
+    NO_PROGRESS(false, false),
+    MADE_PROGRESS(true, false),
+    DONE(true, true),
+    WAS_ALREADY_DONE(false, true);
+
+    private final boolean madeProgress;
+    private final boolean isDone;
+
+    ProgressState(boolean madeProgress, boolean isDone) {
+        this.madeProgress = madeProgress;
+        this.isDone = isDone;
+    }
+
+    public boolean isMadeProgress() {
+        return madeProgress;
+    }
+
+    public boolean isDone() {
+        return isDone;
+    }
+
+    public static ProgressState valueOf(boolean isMadeProgress, boolean 
isDone) {
+        return isDone ? isMadeProgress ? ProgressState.DONE : 
ProgressState.WAS_ALREADY_DONE
+            : isMadeProgress ? ProgressState.MADE_PROGRESS : 
ProgressState.NO_PROGRESS;
+
+    }
+
+}
diff --git 
a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/Task.java
 
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/Task.java
new file mode 100644
index 000000000..688f0476b
--- /dev/null
+++ 
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/Task.java
@@ -0,0 +1,46 @@
+/*
+ * 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.engine.server.execution;
+
+import com.hazelcast.spi.impl.operationservice.OperationService;
+import lombok.NonNull;
+
+import java.io.Serializable;
+
+public interface Task extends Serializable {
+
+    default void init() {
+    }
+
+    @NonNull
+    ProgressState call();
+
+    @NonNull
+    Long getTaskID();
+
+    default boolean isCooperative() {
+        return false;
+    }
+
+    default void close() {
+    }
+
+    default void setOperationService(OperationService operationService) {
+    }
+
+}
diff --git 
a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/TaskExecutionContext.java
 
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/TaskExecutionContext.java
new file mode 100644
index 000000000..3cd8c6734
--- /dev/null
+++ 
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/TaskExecutionContext.java
@@ -0,0 +1,49 @@
+/*
+ * 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.engine.server.execution;
+
+import org.apache.seatunnel.engine.server.TaskExecutionService;
+
+import java.util.concurrent.CompletableFuture;
+
+public class TaskExecutionContext {
+
+    // future which is Task submit
+    public volatile CompletableFuture<Void> executionFuture;
+
+    // future which can only be used to cancel the local execution.
+    private volatile CompletableFuture<Void> cancellationFuture;
+
+    private TaskExecutionService taskExecutionService;
+
+    public TaskExecutionContext(
+        CompletableFuture<Void> executionFuture,
+        CompletableFuture<Void> cancellationFuture,
+        TaskExecutionService taskExecutionService
+    ) {
+        this.executionFuture = executionFuture;
+        this.cancellationFuture = cancellationFuture;
+        this.taskExecutionService = taskExecutionService;
+    }
+
+    public CompletableFuture<Void> cancel() {
+        cancellationFuture.cancel(true);
+        return executionFuture;
+    }
+
+}
diff --git 
a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/TaskGroup.java
 
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/TaskGroup.java
new file mode 100644
index 000000000..fdf88cde7
--- /dev/null
+++ 
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/TaskGroup.java
@@ -0,0 +1,30 @@
+/*
+ * 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.engine.server.execution;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+import java.util.Collection;
+
+@Data
+@AllArgsConstructor
+public class TaskGroup implements Serializable {
+    private final Collection<Task> tasks;
+}
diff --git 
a/seatunnel-engine/seatunnel-engine-server/src/main/test/execution/TaskExecutionServiceTest.java
 
b/seatunnel-engine/seatunnel-engine-server/src/main/test/execution/TaskExecutionServiceTest.java
new file mode 100644
index 000000000..014e62efd
--- /dev/null
+++ 
b/seatunnel-engine/seatunnel-engine-server/src/main/test/execution/TaskExecutionServiceTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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 execution;
+
+import static org.junit.Assert.assertTrue;
+
+import org.apache.seatunnel.engine.server.SeaTunnelNodeContext;
+import org.apache.seatunnel.engine.server.SeaTunnelServer;
+import org.apache.seatunnel.engine.server.execution.TaskExecutionContext;
+import org.apache.seatunnel.engine.server.TaskExecutionService;
+
+import com.hazelcast.config.Config;
+import com.hazelcast.instance.impl.HazelcastInstanceFactory;
+import com.hazelcast.instance.impl.HazelcastInstanceImpl;
+import com.hazelcast.instance.impl.HazelcastInstanceProxy;
+import com.hazelcast.logging.ILogger;
+import org.junit.Test;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.BiConsumer;
+
+
+public class TaskExecutionServiceTest {
+
+    HazelcastInstanceImpl instance = ((HazelcastInstanceProxy) 
HazelcastInstanceFactory.newHazelcastInstance(new Config(), 
Thread.currentThread().getName(), new SeaTunnelNodeContext())).getOriginal();
+    SeaTunnelServer service = 
instance.node.nodeEngine.getService(SeaTunnelServer.SERVICE_NAME);
+    ILogger logger = 
instance.node.nodeEngine.getLogger(TaskExecutionServiceTest.class);
+
+
+    @Test
+    public void testAll() throws InterruptedException {
+        testCancel();
+        testFinish();
+    }
+
+    public void testCancel() throws InterruptedException {
+        TaskExecutionService taskExecutionService = 
service.getTaskExecutionService();
+
+        AtomicBoolean stop = new AtomicBoolean(false);
+        TestTask testTask = new TestTask(stop, logger);
+
+        TaskExecutionContext taskExecutionContext = 
taskExecutionService.submitTask(testTask);
+
+        Thread.sleep(3000);
+
+        taskExecutionContext.cancel();
+
+        Thread.sleep(30000);
+        
assertTrue(taskExecutionContext.executionFuture.isCompletedExceptionally());
+    }
+
+    public void testFinish() throws InterruptedException {
+        TaskExecutionService taskExecutionService = 
service.getTaskExecutionService();
+
+        AtomicBoolean stop = new AtomicBoolean(false);
+        AtomicBoolean futureMark = new AtomicBoolean(false);
+        TestTask testTasklet = new TestTask(stop, logger);
+
+        TaskExecutionContext taskExecutionContext = 
taskExecutionService.submitTask(testTasklet);
+        taskExecutionContext.executionFuture.whenComplete(new BiConsumer<Void, 
Throwable>() {
+            @Override
+            public void accept(Void unused, Throwable throwable) {
+                futureMark.set(true);
+            }
+        });
+
+        Thread.sleep(3000);
+
+        stop.set(true);
+
+        Thread.sleep(1000);
+
+        assertTrue(taskExecutionContext.executionFuture.isDone());
+        assertTrue(futureMark.get());
+    }
+}
\ No newline at end of file
diff --git 
a/seatunnel-engine/seatunnel-engine-server/src/main/test/execution/TestTask.java
 
b/seatunnel-engine/seatunnel-engine-server/src/main/test/execution/TestTask.java
new file mode 100644
index 000000000..21ae87bfe
--- /dev/null
+++ 
b/seatunnel-engine/seatunnel-engine-server/src/main/test/execution/TestTask.java
@@ -0,0 +1,63 @@
+/*
+ * 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 execution;
+
+import org.apache.seatunnel.engine.server.execution.ProgressState;
+import org.apache.seatunnel.engine.server.execution.Task;
+
+import com.hazelcast.logging.ILogger;
+import lombok.NonNull;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * For test use, only print logs
+ */
+public class TestTask implements Task {
+
+    AtomicBoolean stop;
+    private final ILogger logger;
+
+    public TestTask(AtomicBoolean stop, ILogger logger){
+        this.stop = stop;
+        this.logger = logger;
+    }
+
+    @NonNull
+    @Override
+    public ProgressState call() {
+        ProgressState progressState;
+        if (!stop.get()){
+            logger.info("TestTasklet is running");
+            try {
+                Thread.sleep(1000);
+            } catch (InterruptedException e) {
+            }
+            progressState = ProgressState.MADE_PROGRESS;
+        }else {
+            progressState = ProgressState.DONE;
+        }
+        return progressState;
+    }
+
+    @NonNull
+    @Override
+    public Long getTaskID() {
+        return 1L;
+    }
+}

Reply via email to