You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2022/03/21 08:27:06 UTC

[GitHub] [iotdb] JackieTien97 commented on a change in pull request #5294: [IOTDB-2726][IOTDB-2727] Basic interfaces definition of mpp scheduling and memory control

JackieTien97 commented on a change in pull request #5294:
URL: https://github.com/apache/iotdb/pull/5294#discussion_r830835685



##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/schedule/queue/L2PriorityQueue.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.iotdb.db.mpp.schedule.queue;
+
+import java.util.Comparator;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * An efficient subclass of {@link IndexedBlockingQueue} with 2-level priority groups. The
+ * advantages compared to {@link L1PriorityQueue} are that each element in this queue will not be
+ * starved to death by its low sequence order.
+ *
+ * <p>The time complexity of operations are:
+ *
+ * <ul>
+ *   <li><b>{@link #remove(IDIndexedAccessible)} ()}: </b> O(logN).
+ *   <li><b>{@link #push(IDIndexedAccessible)}: </b> O(logN).
+ *   <li><b>{@link #poll()}: </b> O(logN).
+ *   <li><b>{@link #get(ID)}}: </b> O(1).
+ * </ul>
+ */
+public class L2PriorityQueue<E extends IDIndexedAccessible> extends IndexedBlockingQueue<E> {
+
+  // Here we use a map not a set to act as a queue because we need to get the element reference
+  // after it was removed.
+  private SortedMap<E, E> workingElements;
+  private SortedMap<E, E> idleElements;
+
+  /**
+   * Init the queue with max capacity and specified comparator.
+   *
+   * @see IndexedBlockingQueue
+   * @param maxCapacity the max capacity of the queue.
+   * @param comparator the comparator for comparing the elements.
+   * @param queryHolder the query holder instance.
+   * @throws IllegalArgumentException if maxCapacity <= 0.
+   */
+  public L2PriorityQueue(int maxCapacity, Comparator<E> comparator, E queryHolder) {
+    super(maxCapacity, queryHolder);
+    this.workingElements = new TreeMap<>(comparator);
+    this.idleElements = new TreeMap<>(comparator);
+  }
+
+  @Override
+  protected boolean isEmpty() {
+    return workingElements.isEmpty() && idleElements.isEmpty();
+  }
+
+  @Override
+  protected E pollFirst() {
+    if (workingElements.isEmpty()) {
+      SortedMap<E, E> tmp = workingElements;
+      workingElements = idleElements;
+      idleElements = tmp;
+    }
+    return workingElements.remove(workingElements.firstKey());
+  }
+
+  @Override
+  protected void pushToQueue(E element) {
+    workingElements.remove(element);
+    idleElements.put(element, element);

Review comment:
       Why we need to remove it from `workingElements` first?

##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/schedule/FragmentInstanceManager.java
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.iotdb.db.mpp.schedule;
+
+import org.apache.iotdb.commons.exception.StartupException;
+import org.apache.iotdb.commons.service.IService;
+import org.apache.iotdb.commons.service.ServiceType;
+import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.ExecFragmentInstance;
+import org.apache.iotdb.db.mpp.schedule.queue.IndexedBlockingQueue;
+import org.apache.iotdb.db.mpp.schedule.queue.L1PriorityQueue;
+import org.apache.iotdb.db.mpp.schedule.queue.L2PriorityQueue;
+import org.apache.iotdb.db.mpp.schedule.task.FragmentInstanceTask;
+import org.apache.iotdb.db.mpp.schedule.task.FragmentInstanceTaskStatus;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
+/** the manager of fragment instances scheduling */
+public class FragmentInstanceManager implements IFragmentInstanceManager, IService {
+
+  private static final Logger logger = LoggerFactory.getLogger(FragmentInstanceManager.class);
+
+  public static IFragmentInstanceManager getInstance() {
+    return InstanceHolder.instance;
+  }
+
+  private final IndexedBlockingQueue<FragmentInstanceTask> readyQueue;
+  private final IndexedBlockingQueue<FragmentInstanceTask> timeoutQueue;
+  private final Set<FragmentInstanceTask> blockedTasks;
+  private final Map<QueryId, Set<FragmentInstanceTask>> queryMap;
+  private final ITaskScheduler scheduler;
+
+  private static final int MAX_CAPACITY = 1000; // TODO: load from config files
+  private static final int WORKER_THREAD_NUM = 4; // TODO: load from config files
+  private static final int QUERY_TIMEOUT_MS = 10000; // TODO: load from config files or requests
+  private final ThreadGroup workerGroups;
+  private final List<AbstractExecutor> threads;
+
+  public FragmentInstanceManager() {
+    this.readyQueue =
+        new L2PriorityQueue<>(
+            MAX_CAPACITY,
+            new FragmentInstanceTask.SchedulePriorityComparator(),
+            new FragmentInstanceTask());
+    this.timeoutQueue =
+        new L1PriorityQueue<>(
+            MAX_CAPACITY,
+            new FragmentInstanceTask.SchedulePriorityComparator(),
+            new FragmentInstanceTask());
+    this.queryMap = new ConcurrentHashMap<>();
+    this.blockedTasks = Collections.synchronizedSet(new HashSet<>());
+    this.scheduler = new Scheduler();
+    this.workerGroups = new ThreadGroup("ScheduleThreads");
+    this.threads = new ArrayList<>();
+  }
+
+  @Override
+  public void start() throws StartupException {
+    for (int i = 0; i < WORKER_THREAD_NUM; i++) {
+      AbstractExecutor t =
+          new FragmentInstanceTaskExecutor(
+              "Worker-Thread-" + i, workerGroups, readyQueue, scheduler);
+      threads.add(t);
+      t.start();
+    }
+    AbstractExecutor t =
+        new FragmentInstanceTimeoutSentinel(
+            "Sentinel-Thread", workerGroups, timeoutQueue, scheduler);
+    threads.add(t);
+    t.start();
+  }
+
+  @Override
+  public void stop() {
+    this.threads.forEach(
+        t -> {
+          try {
+            t.close();
+          } catch (IOException e) {
+            // Only a field is set, there's no chance to throw an IOException
+          }
+        });
+  }
+
+  @Override
+  public ServiceType getID() {
+    return ServiceType.FRAGMENT_INSTANCE_MANAGER_SERVICE;
+  }
+
+  @Override
+  public void submitFragmentInstances(QueryId queryId, List<ExecFragmentInstance> instances) {
+    Set<FragmentInstanceTask> tasks =
+        instances.stream()
+            .map(
+                v ->
+                    new FragmentInstanceTask(v, QUERY_TIMEOUT_MS, FragmentInstanceTaskStatus.READY))
+            .collect(Collectors.toSet());
+    queryMap.put(queryId, Collections.synchronizedSet(tasks));

Review comment:
       One query may call `submitFragmentInstances` many times, so `put` here may not be right. `computeIfAbsent` plus `add` will be better.

##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/schedule/FragmentInstanceManager.java
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.iotdb.db.mpp.schedule;
+
+import org.apache.iotdb.commons.exception.StartupException;
+import org.apache.iotdb.commons.service.IService;
+import org.apache.iotdb.commons.service.ServiceType;
+import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.ExecFragmentInstance;
+import org.apache.iotdb.db.mpp.schedule.queue.IndexedBlockingQueue;
+import org.apache.iotdb.db.mpp.schedule.queue.L1PriorityQueue;
+import org.apache.iotdb.db.mpp.schedule.queue.L2PriorityQueue;
+import org.apache.iotdb.db.mpp.schedule.task.FragmentInstanceTask;
+import org.apache.iotdb.db.mpp.schedule.task.FragmentInstanceTaskStatus;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
+/** the manager of fragment instances scheduling */
+public class FragmentInstanceManager implements IFragmentInstanceManager, IService {
+
+  private static final Logger logger = LoggerFactory.getLogger(FragmentInstanceManager.class);
+
+  public static IFragmentInstanceManager getInstance() {
+    return InstanceHolder.instance;
+  }
+
+  private final IndexedBlockingQueue<FragmentInstanceTask> readyQueue;
+  private final IndexedBlockingQueue<FragmentInstanceTask> timeoutQueue;
+  private final Set<FragmentInstanceTask> blockedTasks;
+  private final Map<QueryId, Set<FragmentInstanceTask>> queryMap;
+  private final ITaskScheduler scheduler;
+
+  private static final int MAX_CAPACITY = 1000; // TODO: load from config files
+  private static final int WORKER_THREAD_NUM = 4; // TODO: load from config files
+  private static final int QUERY_TIMEOUT_MS = 10000; // TODO: load from config files or requests
+  private final ThreadGroup workerGroups;
+  private final List<AbstractExecutor> threads;
+
+  public FragmentInstanceManager() {
+    this.readyQueue =
+        new L2PriorityQueue<>(
+            MAX_CAPACITY,
+            new FragmentInstanceTask.SchedulePriorityComparator(),
+            new FragmentInstanceTask());
+    this.timeoutQueue =
+        new L1PriorityQueue<>(
+            MAX_CAPACITY,
+            new FragmentInstanceTask.SchedulePriorityComparator(),
+            new FragmentInstanceTask());
+    this.queryMap = new ConcurrentHashMap<>();
+    this.blockedTasks = Collections.synchronizedSet(new HashSet<>());
+    this.scheduler = new Scheduler();
+    this.workerGroups = new ThreadGroup("ScheduleThreads");
+    this.threads = new ArrayList<>();
+  }
+
+  @Override
+  public void start() throws StartupException {
+    for (int i = 0; i < WORKER_THREAD_NUM; i++) {
+      AbstractExecutor t =
+          new FragmentInstanceTaskExecutor(
+              "Worker-Thread-" + i, workerGroups, readyQueue, scheduler);
+      threads.add(t);
+      t.start();
+    }
+    AbstractExecutor t =
+        new FragmentInstanceTimeoutSentinel(
+            "Sentinel-Thread", workerGroups, timeoutQueue, scheduler);
+    threads.add(t);
+    t.start();
+  }
+
+  @Override
+  public void stop() {
+    this.threads.forEach(
+        t -> {
+          try {
+            t.close();
+          } catch (IOException e) {
+            // Only a field is set, there's no chance to throw an IOException
+          }
+        });
+  }
+
+  @Override
+  public ServiceType getID() {
+    return ServiceType.FRAGMENT_INSTANCE_MANAGER_SERVICE;
+  }
+
+  @Override
+  public void submitFragmentInstances(QueryId queryId, List<ExecFragmentInstance> instances) {
+    Set<FragmentInstanceTask> tasks =
+        instances.stream()
+            .map(
+                v ->
+                    new FragmentInstanceTask(v, QUERY_TIMEOUT_MS, FragmentInstanceTaskStatus.READY))
+            .collect(Collectors.toSet());
+    queryMap.put(queryId, Collections.synchronizedSet(tasks));
+    for (FragmentInstanceTask task : tasks) {
+      task.lock();

Review comment:
       Why we need task.lock() here, it seems that `queue.push()` won't cause some race condition.

##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/schedule/AbstractExecutor.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.iotdb.db.mpp.schedule;
+
+import org.apache.iotdb.db.mpp.schedule.queue.IndexedBlockingQueue;
+import org.apache.iotdb.db.mpp.schedule.task.FragmentInstanceTask;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+
+/** an abstract executor for {@link FragmentInstanceTask} */
+public abstract class AbstractExecutor extends Thread implements Closeable {
+
+  private static final Logger logger = LoggerFactory.getLogger(AbstractExecutor.class);
+  private final IndexedBlockingQueue<FragmentInstanceTask> queue;
+  private final ITaskScheduler scheduler;

Review comment:
       Why not change the `private` to `protected`, so that `FragmentInstanceTaskExecutor` will avoid calling `getScheduler()` method all the time.




-- 
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: reviews-unsubscribe@iotdb.apache.org

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