You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@seatunnel.apache.org by GitBox <gi...@apache.org> on 2022/08/31 02:58:46 UTC

[GitHub] [incubator-seatunnel] EricJoy2048 commented on a diff in pull request #2472: [Engine][ResourceManager] Add ResourceManager

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


##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/AbstractResourceManager.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.resourcemanager;
+
+import org.apache.seatunnel.engine.common.runtime.ExecutionMode;
+import org.apache.seatunnel.engine.server.SeaTunnelServer;
+import org.apache.seatunnel.engine.server.resourcemanager.heartbeat.HeartbeatListener;
+import org.apache.seatunnel.engine.server.resourcemanager.heartbeat.HeartbeatManager;
+import org.apache.seatunnel.engine.server.resourcemanager.opeartion.ReleaseSlotOperation;
+import org.apache.seatunnel.engine.server.resourcemanager.opeartion.RequestSlotOperation;
+import org.apache.seatunnel.engine.server.resourcemanager.resource.ResourceProfile;
+import org.apache.seatunnel.engine.server.resourcemanager.resource.SlotProfile;
+import org.apache.seatunnel.engine.server.resourcemanager.worker.WorkerProfile;
+import org.apache.seatunnel.engine.server.service.slot.SlotAndWorkerProfile;
+
+import com.google.common.collect.Lists;
+import com.hazelcast.cluster.Address;
+import com.hazelcast.logging.ILogger;
+import com.hazelcast.logging.Logger;
+import com.hazelcast.spi.impl.NodeEngine;
+import com.hazelcast.spi.impl.operationservice.InvocationBuilder;
+import com.hazelcast.spi.impl.operationservice.Operation;
+import com.hazelcast.spi.impl.operationservice.impl.InvocationFuture;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public abstract class AbstractResourceManager implements ResourceManager {
+
+    private static final long DEFAULT_WORKER_CHECK_INTERVAL = 500;
+    private static final ILogger LOGGER = Logger.getLogger(AbstractResourceManager.class);
+
+    protected final Map<String, WorkerProfile> registerWorker;
+
+    private final HeartbeatManager heartbeatManager;
+    private final NodeEngine nodeEngine;
+
+    private final ExecutionMode mode = ExecutionMode.LOCAL;
+
+    public AbstractResourceManager(NodeEngine nodeEngine) {
+        this.registerWorker = new ConcurrentHashMap<>();
+        this.nodeEngine = nodeEngine;
+        this.heartbeatManager = new HeartbeatManager(new WorkerHeartbeatListener());
+    }
+
+    @Override
+    public void init() {
+        heartbeatManager.start(Executors.newSingleThreadScheduledExecutor());
+    }
+
+    @Override
+    public CompletableFuture<SlotProfile> applyResource(long jobId, ResourceProfile resourceProfile) throws NoEnoughResourceException {
+        waitingWorkerRegister();
+        CompletableFuture<SlotProfile> completableFuture = new CompletableFuture<>();
+        applyResources(jobId, Collections.singletonList(resourceProfile)).whenComplete((profile, error) -> {
+            if (error != null) {
+                completableFuture.completeExceptionally(error);
+            } else {
+                completableFuture.complete(profile.get(0));
+            }
+        });
+        return completableFuture;
+    }
+
+    private void waitingWorkerRegister() {
+        if (ExecutionMode.LOCAL.equals(mode)) {
+            // Local mode, should wait worker(master node) register.
+            try {
+                while (registerWorker.isEmpty()) {
+                    Thread.sleep(DEFAULT_WORKER_CHECK_INTERVAL);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    @Override
+    public CompletableFuture<List<SlotProfile>> applyResources(long jobId,
+                                                               List<ResourceProfile> resourceProfile) throws NoEnoughResourceException {
+        // TODO Apply for the profile with the largest resources first, and then decrease in order. In this
+        //  way, the success rate of resource application will be higher.
+        waitingWorkerRegister();
+        CompletableFuture<List<SlotProfile>> completableFuture = new CompletableFuture<>();
+        AtomicInteger successCount = new AtomicInteger();
+        Map<Integer, SlotProfile> slotProfiles = new ConcurrentHashMap<>();
+        List<CompletableFuture<SlotAndWorkerProfile>> allRequestFuture = new ArrayList<>();
+        for (int i = 0; i < resourceProfile.size(); i++) {
+            ResourceProfile r = resourceProfile.get(i);
+            Optional<WorkerProfile> workerProfile =

Review Comment:
   Can add some comments what is the different between 
   
   ```
   registerWorker.values().stream().filter(worker -> worker.getUnassignedResource().enoughThan(r)).findAny();
   ```
   
   and 
   
   ```
   registerWorker.values().stream().filter(worker -> Arrays.stream(worker.getUnassignedSlots()).anyMatch(slot -> slot.getResourceProfile().enoughThan(r))).findAny();
   ```



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalVertex.java:
##########
@@ -188,29 +194,42 @@ public void deploy(@NonNull Address address) {
                                 this.getTaskFullName(), executionState.get()))));
                     }
                 }
+                monitorTask(completeFuture);
             } else if (ExecutionState.CANCELING.equals(this.getExecutionState().get())) {
                 turnToEndState(ExecutionState.CANCELED);
                 taskFuture.complete(new TaskExecutionState(this.taskGroupLocation, executionState.get(), null));
             } else {
                 turnToEndState(ExecutionState.FAILED);
                 taskFuture.complete(new TaskExecutionState(this.taskGroupLocation, executionState.get(),
-                    new JobException(String.format("%s turn to a unexpected state"))));
+                    new JobException(String.format("%s turn to a unexpected state", jobImmutableInformation.getJobId()))));

Review Comment:
   We can only know the state of this PhysicalVertex turn to an unexpected state, not the Job.



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/AbstractResourceManager.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.resourcemanager;
+
+import org.apache.seatunnel.engine.common.runtime.ExecutionMode;
+import org.apache.seatunnel.engine.server.SeaTunnelServer;
+import org.apache.seatunnel.engine.server.resourcemanager.heartbeat.HeartbeatListener;
+import org.apache.seatunnel.engine.server.resourcemanager.heartbeat.HeartbeatManager;
+import org.apache.seatunnel.engine.server.resourcemanager.opeartion.ReleaseSlotOperation;
+import org.apache.seatunnel.engine.server.resourcemanager.opeartion.RequestSlotOperation;
+import org.apache.seatunnel.engine.server.resourcemanager.resource.ResourceProfile;
+import org.apache.seatunnel.engine.server.resourcemanager.resource.SlotProfile;
+import org.apache.seatunnel.engine.server.resourcemanager.worker.WorkerProfile;
+import org.apache.seatunnel.engine.server.service.slot.SlotAndWorkerProfile;
+
+import com.google.common.collect.Lists;
+import com.hazelcast.cluster.Address;
+import com.hazelcast.logging.ILogger;
+import com.hazelcast.logging.Logger;
+import com.hazelcast.spi.impl.NodeEngine;
+import com.hazelcast.spi.impl.operationservice.InvocationBuilder;
+import com.hazelcast.spi.impl.operationservice.Operation;
+import com.hazelcast.spi.impl.operationservice.impl.InvocationFuture;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public abstract class AbstractResourceManager implements ResourceManager {
+
+    private static final long DEFAULT_WORKER_CHECK_INTERVAL = 500;
+    private static final ILogger LOGGER = Logger.getLogger(AbstractResourceManager.class);
+
+    protected final Map<String, WorkerProfile> registerWorker;
+
+    private final HeartbeatManager heartbeatManager;
+    private final NodeEngine nodeEngine;
+
+    private final ExecutionMode mode = ExecutionMode.LOCAL;
+
+    public AbstractResourceManager(NodeEngine nodeEngine) {
+        this.registerWorker = new ConcurrentHashMap<>();
+        this.nodeEngine = nodeEngine;
+        this.heartbeatManager = new HeartbeatManager(new WorkerHeartbeatListener());
+    }
+
+    @Override
+    public void init() {
+        heartbeatManager.start(Executors.newSingleThreadScheduledExecutor());
+    }
+
+    @Override
+    public CompletableFuture<SlotProfile> applyResource(long jobId, ResourceProfile resourceProfile) throws NoEnoughResourceException {
+        waitingWorkerRegister();
+        CompletableFuture<SlotProfile> completableFuture = new CompletableFuture<>();
+        applyResources(jobId, Collections.singletonList(resourceProfile)).whenComplete((profile, error) -> {
+            if (error != null) {
+                completableFuture.completeExceptionally(error);
+            } else {
+                completableFuture.complete(profile.get(0));
+            }
+        });
+        return completableFuture;
+    }
+
+    private void waitingWorkerRegister() {
+        if (ExecutionMode.LOCAL.equals(mode)) {
+            // Local mode, should wait worker(master node) register.
+            try {
+                while (registerWorker.isEmpty()) {
+                    Thread.sleep(DEFAULT_WORKER_CHECK_INTERVAL);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    @Override
+    public CompletableFuture<List<SlotProfile>> applyResources(long jobId,
+                                                               List<ResourceProfile> resourceProfile) throws NoEnoughResourceException {
+        // TODO Apply for the profile with the largest resources first, and then decrease in order. In this
+        //  way, the success rate of resource application will be higher.
+        waitingWorkerRegister();
+        CompletableFuture<List<SlotProfile>> completableFuture = new CompletableFuture<>();
+        AtomicInteger successCount = new AtomicInteger();
+        Map<Integer, SlotProfile> slotProfiles = new ConcurrentHashMap<>();
+        List<CompletableFuture<SlotAndWorkerProfile>> allRequestFuture = new ArrayList<>();
+        for (int i = 0; i < resourceProfile.size(); i++) {
+            ResourceProfile r = resourceProfile.get(i);
+            Optional<WorkerProfile> workerProfile =
+                registerWorker.values().stream().filter(worker -> worker.getUnassignedResource().enoughThan(r)).findAny();
+
+            if (!workerProfile.isPresent()) {
+                workerProfile =
+                    registerWorker.values().stream().filter(worker -> Arrays.stream(worker.getUnassignedSlots()).anyMatch(slot -> slot.getResourceProfile().enoughThan(r))).findAny();
+            }
+
+            if (workerProfile.isPresent()) {
+                InvocationFuture<SlotAndWorkerProfile> future = sendToMember(new RequestSlotOperation(jobId, r), workerProfile.get().getAddress());
+                int finalI = i;
+                CompletableFuture<SlotAndWorkerProfile> internalCompletableFuture = future.whenComplete(
+                    (slotAndWorkerProfile, error) -> {
+                        if (error != null) {
+                            throw new RuntimeException(error);
+                        } else {
+                            workerTouch(slotAndWorkerProfile.getWorkerProfile());
+                            if (null != slotAndWorkerProfile.getSlotProfile()) {
+                                slotProfiles.put(finalI, slotAndWorkerProfile.getSlotProfile());
+                                if (successCount.incrementAndGet() == resourceProfile.size()) {
+                                    completableFuture.complete(Lists.newArrayList(slotProfiles.values()));
+                                }
+                            } else {
+                                slotProfiles.put(finalI, null);
+                            }
+                        }
+                    }
+                );
+                allRequestFuture.add(internalCompletableFuture);
+            } else {
+                CompletableFuture.allOf(allRequestFuture.toArray(new CompletableFuture[0])).whenComplete((unused, error) -> {
+                    releaseAllResourceInternal(jobId, slotProfiles.values());
+                        if (error != null) {
+                            completableFuture.completeExceptionally(error);
+                        } else {
+                            completableFuture.completeExceptionally(new NoEnoughResourceException("can't apply resource request: " + r));
+                        }
+                    }
+                );
+                return completableFuture;
+            }
+        }
+        CompletableFuture.allOf(allRequestFuture.toArray(new CompletableFuture[0])).whenComplete((unused, error) -> {

Review Comment:
   `new CompletableFuture[0]` ?



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org