EricJoy2048 commented on code in PR #2386:
URL: 
https://github.com/apache/incubator-seatunnel/pull/2386#discussion_r944406116


##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/config/FlowConfig.java:
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.dag.physical.config;
+
+import java.io.Serializable;
+
+/**
+ * This interface indicates that this class is the configuration information 
of Flow
+ */
+public interface FlowConfig extends Serializable {

Review Comment:
   Can you introduce the scenario of using this interface?



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/serializable/TaskDataSerializerHook.java:
##########
@@ -57,14 +73,24 @@ private static class Factory implements 
DataSerializableFactory {
         @Override
         public IdentifiedDataSerializable create(int typeId) {
             switch (typeId) {
-                case REGISTER_TYPE:
-                    return new RegisterOperation();
+                case SOURCE_REGISTER_TYPE:
+                    return new SourceRegisterOperation();
                 case REQUEST_SPLIT_TYPE:
                     return new RequestSplitOperation();
                 case ASSIGN_SPLIT_TYPE:
-                    return new AssignSplitOperation();
+                    return new AssignSplitOperation<>();
                 case TASK_GROUP_INFO_TYPE:
                     return new TaskGroupImmutableInformation();
+                case SOURCE_UNREGISTER_TYPE:
+                    return new SourceUnregisterOperation();
+                case SINK_REGISTER_TYPE:
+                    return new SinkRegisterOperation();
+                case SINK_UNREGISTER_TYPE:
+                    return new SinkUnregisterOperation();
+                case TASK_LOCATION_TYPE:
+                    return new TaskLocation();
+                case PROGRESS_TYPE:
+                    return new Progress();
                 default:
                     return null;

Review Comment:
   I suggest throw a `throw new IllegalArgumentException("Unknown type id " + 
typeId);` here.



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/operation/DeployTaskOperation.java:
##########
@@ -42,8 +42,8 @@ public DeployTaskOperation(@NonNull Data 
taskImmutableInformation) {
 
     @Override
     protected NonCompletableFuture<?> doRun() throws Exception {
-        TaskExecutionService taskExecutionService = getService();
-        NonCompletableFuture<TaskExecutionState> voidCompletableFuture = 
taskExecutionService.deployTask(taskImmutableInformation);
+        SeaTunnelServer server = getService();
+        NonCompletableFuture<TaskExecutionState> voidCompletableFuture =  
server.getTaskExecutionService().deployTask(taskImmutableInformation);
         return voidCompletableFuture;

Review Comment:
   ```suggestion
   ```



##########
seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/dag/PhysicalPlanTest.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.dag;
+
+import org.apache.seatunnel.connectors.seatunnel.console.sink.ConsoleSink;
+import org.apache.seatunnel.connectors.seatunnel.fake.source.FakeSource;
+import org.apache.seatunnel.engine.common.Constant;
+import org.apache.seatunnel.engine.common.config.SeaTunnelConfig;
+import org.apache.seatunnel.engine.common.utils.IdGenerator;
+import org.apache.seatunnel.engine.core.dag.actions.Action;
+import org.apache.seatunnel.engine.core.dag.actions.SinkAction;
+import org.apache.seatunnel.engine.core.dag.actions.SourceAction;
+import org.apache.seatunnel.engine.core.dag.logical.LogicalDag;
+import org.apache.seatunnel.engine.core.dag.logical.LogicalEdge;
+import org.apache.seatunnel.engine.core.dag.logical.LogicalVertex;
+import org.apache.seatunnel.engine.core.job.JobImmutableInformation;
+import org.apache.seatunnel.engine.server.SeaTunnelNodeContext;
+import org.apache.seatunnel.engine.server.dag.physical.PhysicalPlan;
+import org.apache.seatunnel.engine.server.dag.physical.PhysicalPlanUtils;
+
+import com.hazelcast.config.Config;
+import com.hazelcast.core.Hazelcast;
+import com.hazelcast.instance.impl.HazelcastInstanceFactory;
+import com.hazelcast.instance.impl.HazelcastInstanceImpl;
+import com.hazelcast.instance.impl.HazelcastInstanceProxy;
+import com.hazelcast.spi.impl.NodeEngine;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.Collections;
+import java.util.concurrent.Executors;
+
+public class PhysicalPlanTest {
+
+    @Test
+    public void testLogicalToPhysical() throws MalformedURLException {
+
+        HazelcastInstanceImpl instance = ((HazelcastInstanceProxy) 
HazelcastInstanceFactory.newHazelcastInstance(new Config(), 
Thread.currentThread().getName(), new SeaTunnelNodeContext(new 
SeaTunnelConfig()))).getOriginal();
+        NodeEngine nodeEngine = instance.node.nodeEngine;
+
+        IdGenerator idGenerator = new IdGenerator();
+
+        Action fake = new SourceAction<>(idGenerator.getNextId(), "fake", new 
FakeSource(),
+                Collections.singletonList(new URL("file:///fake.jar")));

Review Comment:
   Where can we find `fake.jar` ?



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java:
##########
@@ -30,48 +50,95 @@
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
-import java.util.UUID;
 
-public class SeaTunnelTask extends AbstractTask {
+public abstract class SeaTunnelTask extends AbstractTask {
 
     private static final long serialVersionUID = 2604309561613784425L;
     private final Flow executionFlow;
 
-    // TODO init memberID in task execution service
-    private UUID memberID = UUID.randomUUID();
-    private int enumeratorTaskID = -1;
+    protected FlowLifeCycle startFlowLifeCycle;
 
-    public SeaTunnelTask(long taskID, Flow executionFlow) {
-        super(taskID);
-        // TODO add enumerator task ID
-        enumeratorTaskID = 1;
+    protected List<OneInputFlowLifeCycle<Record>> outputs;
+
+    protected int indexID;
+
+    private TaskGroup taskBelongGroup;
+
+    public SeaTunnelTask(long jobID, TaskLocation taskID, int indexID, Flow 
executionFlow) {
+        super(jobID, taskID);
+        this.indexID = indexID;
         this.executionFlow = executionFlow;
     }
 
     @Override
-    public void init() {
+    public void init() throws Exception {
+        super.init();
+        startFlowLifeCycle = convertFlowToActionLifeCycle(executionFlow);
     }
 
-    @Override
-    public void close() throws IOException {
-        super.close();
+    public void setTaskGroup(TaskGroup group) {
+        this.taskBelongGroup = group;
     }
 
-    private void register() {
-        if (startFromSource()) {
-            this.executionContext.sendToMaster(new RegisterOperation(taskID, 
enumeratorTaskID));
+    @SuppressWarnings({"unchecked", "rawtypes", "checkstyle:MagicNumber"})
+    private FlowLifeCycle convertFlowToActionLifeCycle(Flow flow) throws 
Exception {

Review Comment:
   I suggest add `@NonNull` if the params must not be null.



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SinkAggregatedCommitterTask.java:
##########
@@ -17,20 +17,64 @@
 
 package org.apache.seatunnel.engine.server.task;
 
+import org.apache.seatunnel.api.sink.SinkAggregatedCommitter;
 import org.apache.seatunnel.engine.core.dag.actions.SinkAction;
+import org.apache.seatunnel.engine.server.execution.TaskLocation;
+
+import com.hazelcast.cluster.Address;
+import com.hazelcast.logging.ILogger;
+import com.hazelcast.logging.Logger;
 
 import java.net.URL;
+import java.util.Arrays;
 import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
 
-public class SinkAggregatedCommitterTask extends CoordinatorTask {
+public class SinkAggregatedCommitterTask<AggregatedCommitInfoT> extends 
CoordinatorTask {
 
+    private static final ILogger LOGGER = 
Logger.getLogger(SinkAggregatedCommitterTask.class);
     private static final long serialVersionUID = 5906594537520393503L;
-    private final SinkAction<?, ?, ?, ?> sink;
+    private final SinkAction<?, ?, ?, AggregatedCommitInfoT> sink;
+
+    private final SinkAggregatedCommitter<?, AggregatedCommitInfoT> 
aggregatedCommitter;
+
+    private final Map<Long, Address> writerAddressMap;
+
+    private final Map<Long, List<AggregatedCommitInfoT>> 
checkpointCommitInfoMap;
 
-    public SinkAggregatedCommitterTask(long taskID, SinkAction<?, ?, ?, ?> 
sink) {
-        super(taskID);
+    public SinkAggregatedCommitterTask(long jobID, TaskLocation taskID, 
SinkAction<?, ?, ?, AggregatedCommitInfoT> sink,
+                                       SinkAggregatedCommitter<?, 
AggregatedCommitInfoT> aggregatedCommitter) {
+        super(jobID, taskID);
         this.sink = sink;
+        this.aggregatedCommitter = aggregatedCommitter;
+        this.writerAddressMap = new ConcurrentHashMap<>();
+        this.checkpointCommitInfoMap = new ConcurrentHashMap<>();
+    }
+
+    @Override
+    public void init() throws Exception {
+        super.init();
+        LOGGER.info("starting seatunnel source split enumerator task, sink 
name: " + sink.getName());

Review Comment:
   `source split enumerator task` shouldn't be here. 



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/SimpleResourceManager.java:
##########
@@ -44,7 +44,7 @@ public Address applyForResource(@NonNull Long jobId, @NonNull 
Long taskId) {
             }
 
             Address localhost =
-                jobAddressMap.putIfAbsent(taskId, new Address("localhost", 
5801));
+                jobAddressMap.putIfAbsent(taskId, new Address("192.168.2.1", 
5701));

Review Comment:
   `192.168.2.1` can not be use here.



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/operation/DeployTaskOperation.java:
##########
@@ -42,8 +42,8 @@ public DeployTaskOperation(@NonNull Data 
taskImmutableInformation) {
 
     @Override
     protected NonCompletableFuture<?> doRun() throws Exception {
-        TaskExecutionService taskExecutionService = getService();
-        NonCompletableFuture<TaskExecutionState> voidCompletableFuture = 
taskExecutionService.deployTask(taskImmutableInformation);
+        SeaTunnelServer server = getService();
+        NonCompletableFuture<TaskExecutionState> voidCompletableFuture =  
server.getTaskExecutionService().deployTask(taskImmutableInformation);

Review Comment:
   ```suggestion
           return 
server.getTaskExecutionService().deployTask(taskImmutableInformation);
   ```



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/TransformSeaTunnelTask.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.task;
+
+import org.apache.seatunnel.api.table.type.Record;
+import org.apache.seatunnel.api.transform.Collector;
+import org.apache.seatunnel.engine.core.dag.actions.SourceAction;
+import org.apache.seatunnel.engine.server.dag.physical.config.SourceConfig;
+import org.apache.seatunnel.engine.server.dag.physical.flow.Flow;
+import org.apache.seatunnel.engine.server.execution.ProgressState;
+import org.apache.seatunnel.engine.server.execution.TaskLocation;
+import org.apache.seatunnel.engine.server.task.flow.OneOutputFlowLifeCycle;
+import org.apache.seatunnel.engine.server.task.flow.SourceFlowLifeCycle;
+
+import com.hazelcast.logging.ILogger;
+import com.hazelcast.logging.Logger;
+import lombok.NonNull;
+
+public class TransformSeaTunnelTask extends SeaTunnelTask {
+
+    private static final ILogger LOGGER = 
Logger.getLogger(TransformSeaTunnelTask.class);
+
+    public TransformSeaTunnelTask(long jobID, TaskLocation taskID, int 
indexID, Flow executionFlow) {
+        super(jobID, taskID, indexID, executionFlow);
+    }
+
+    private Collector<Record> collector;
+
+    @Override
+    public void init() throws Exception {
+        super.init();
+        LOGGER.info("starting seatunnel transform task, index " + indexID);
+        collector = new SeaTunnelTransformCollector<>(outputs);
+    }
+
+    @Override
+    protected SourceFlowLifeCycle<?, ?> 
createSourceFlowLifeCycle(SourceAction<?, ?, ?> sourceAction,
+                                                                  SourceConfig 
config) {
+        throw new UnsupportedOperationException("MiddleSeaTunnelTask can't 
create SourceFlowLifeCycle");
+    }
+
+    @NonNull
+    @Override
+    @SuppressWarnings("unchecked")
+    public ProgressState call() throws Exception {
+        if (startFlowLifeCycle instanceof OneOutputFlowLifeCycle) {

Review Comment:
   I suggest we check the really type before the task begin running and read 
data. We should try our best to avoid Exceptions when processing data. 
Especially when these exceptions can be checked before processing data.



-- 
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]

Reply via email to