You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ja...@apache.org on 2023/01/10 00:50:18 UTC
[iotdb] branch master updated: [IOTDB-4005] Pipeline fragmentInstance to implement parallel execution
This is an automated email from the ASF dual-hosted git repository.
jackietien pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iotdb.git
The following commit(s) were added to refs/heads/master by this push:
new c085fb1a8c [IOTDB-4005] Pipeline fragmentInstance to implement parallel execution
c085fb1a8c is described below
commit c085fb1a8c8319093431e58cc6bcb82c9b2a6a8d
Author: Xiangwei Wei <34...@users.noreply.github.com>
AuthorDate: Tue Jan 10 08:50:10 2023 +0800
[IOTDB-4005] Pipeline fragmentInstance to implement parallel execution
---
.../iotdb/db/mpp/execution/StateMachine.java | 4 +-
.../iotdb/db/mpp/execution/driver/DataDriver.java | 37 +-
.../db/mpp/execution/driver/DataDriverContext.java | 48 ++-
.../iotdb/db/mpp/execution/driver/Driver.java | 34 +-
.../db/mpp/execution/driver/DriverContext.java | 76 +++-
.../iotdb/db/mpp/execution/driver/IDriver.java | 5 +-
.../db/mpp/execution/driver/SchemaDriver.java | 5 +-
.../mpp/execution/driver/SchemaDriverContext.java | 13 +-
.../exchange/IMPPDataExchangeManager.java | 4 +-
.../db/mpp/execution/exchange/LocalSinkHandle.java | 25 +-
.../mpp/execution/exchange/LocalSourceHandle.java | 27 +-
.../execution/exchange/MPPDataExchangeManager.java | 113 ++++-
.../mpp/execution/exchange/SharedTsBlockQueue.java | 5 +
.../db/mpp/execution/exchange/SinkHandle.java | 2 +-
.../db/mpp/execution/exchange/SourceHandle.java | 2 +-
.../fragment/FragmentInstanceContext.java | 127 +++---
.../fragment/FragmentInstanceExecution.java | 25 +-
.../fragment/FragmentInstanceManager.java | 37 +-
.../db/mpp/execution/operator/OperatorContext.java | 32 +-
.../factory/OperatorFactory.java} | 30 +-
.../factory/SourceOperatorFactory.java} | 26 +-
.../operator/process/join/TimeJoinOperator.java | 1 -
.../last/AbstractUpdateLastCacheOperator.java | 2 +-
.../schema/CountGroupByLevelScanOperator.java | 3 +-
...rator.java => NodePathsSchemaScanOperator.java} | 147 +++----
.../operator/schema/SchemaCountOperator.java | 3 +-
.../operator/schema/SchemaQueryScanOperator.java | 5 +-
.../operator/source/AlignedSeriesScanUtil.java | 3 +-
.../operator/source/ExchangeOperator.java | 4 +
.../operator/source/SeriesScanOperator.java | 76 +++-
.../execution/operator/source/SeriesScanUtil.java | 6 +-
.../execution/schedule/AbstractDriverThread.java | 9 +-
.../db/mpp/execution/schedule/DriverScheduler.java | 143 ++++---
...eption.java => DriverTaskAbortedException.java} | 8 +-
.../mpp/execution/schedule/DriverTaskThread.java | 12 +-
.../schedule/DriverTaskTimeoutSentinelThread.java | 2 +-
.../mpp/execution/schedule/IDriverScheduler.java | 7 +-
.../schedule/queue/IDIndexedAccessible.java | 2 +-
.../schedule/queue/IndexedBlockingQueue.java | 3 +-
.../execution/schedule/queue/L1PriorityQueue.java | 10 +-
.../execution/schedule/queue/L2PriorityQueue.java | 16 +-
.../db/mpp/execution/schedule/task/DriverTask.java | 44 +-
.../task/{DriverTaskID.java => DriverTaskId.java} | 49 ++-
.../db/mpp/plan/execution/QueryExecution.java | 2 +-
.../plan/planner/LocalExecutionPlanContext.java | 175 +++++---
.../db/mpp/plan/planner/LocalExecutionPlanner.java | 100 ++---
.../plan/planner/MemoryDistributionCalculator.java | 472 ---------------------
.../db/mpp/plan/planner/OperatorTreeGenerator.java | 269 +++++++-----
.../db/mpp/plan/planner/PipelineDriverFactory.java | 59 +++
.../iotdb/db/mpp/execution/DataDriverTest.java | 52 +--
.../execution/exchange/LocalSinkHandleTest.java | 26 +-
.../execution/exchange/LocalSourceHandleTest.java | 12 +-
.../exchange/MPPDataExchangeManagerTest.java | 8 +-
.../operator/AggregationOperatorTest.java | 17 +-
.../AlignedSeriesAggregationScanOperatorTest.java | 9 +-
.../operator/AlignedSeriesScanOperatorTest.java | 92 ++--
.../operator/DeviceMergeOperatorTest.java | 77 ++--
.../execution/operator/DeviceViewOperatorTest.java | 16 +-
.../mpp/execution/operator/FillOperatorTest.java | 17 +-
.../operator/LastQueryMergeOperatorTest.java | 25 +-
.../execution/operator/LastQueryOperatorTest.java | 49 +--
.../operator/LastQuerySortOperatorTest.java | 49 +--
.../mpp/execution/operator/LimitOperatorTest.java | 22 +-
.../execution/operator/LinearFillOperatorTest.java | 57 +--
.../execution/operator/MergeSortOperatorTest.java | 205 ++++-----
.../mpp/execution/operator/OffsetOperatorTest.java | 73 ++--
.../mpp/execution/operator/OperatorMemoryTest.java | 46 +-
.../operator/RawDataAggregationOperatorTest.java | 22 +-
.../SeriesAggregationScanOperatorTest.java | 8 +-
.../execution/operator/SeriesScanOperatorTest.java | 8 +-
.../operator/SingleDeviceViewOperatorTest.java | 20 +-
.../SlidingWindowAggregationOperatorTest.java | 12 +-
.../execution/operator/TimeJoinOperatorTest.java | 56 ++-
.../operator/UpdateLastCacheOperatorTest.java | 13 +-
.../operator/VerticallyConcatOperatorTest.java | 14 +-
.../schema/CountGroupByLevelMergeOperatorTest.java | 15 +-
.../operator/schema/SchemaCountOperatorTest.java | 23 +-
.../schema/SchemaQueryScanOperatorTest.java | 25 +-
.../schedule/DefaultDriverSchedulerTest.java | 166 +++++---
.../execution/schedule/DriverSchedulerTest.java | 45 +-
.../DriverTaskTimeoutSentinelThreadTest.java | 23 +-
.../schedule/queue/L1PriorityQueueTest.java | 6 +-
.../schedule/queue/L2PriorityQueueTest.java | 12 +-
.../mpp/execution/schedule/queue/QueueElement.java | 4 +-
84 files changed, 1813 insertions(+), 1820 deletions(-)
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/StateMachine.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/StateMachine.java
index cd85613b55..c4482b87eb 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/StateMachine.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/StateMachine.java
@@ -269,12 +269,10 @@ public class StateMachine<T> {
public void addStateChangeListener(StateChangeListener<T> stateChangeListener) {
requireNonNull(stateChangeListener, "stateChangeListener is null");
- boolean inTerminalState;
T currentState;
synchronized (lock) {
currentState = state;
- inTerminalState = isTerminalState(currentState);
- if (!inTerminalState) {
+ if (!isTerminalState(currentState)) {
stateChangeListeners.add(stateChangeListener);
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/DataDriver.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/DataDriver.java
index 6dac787763..2153af9e74 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/DataDriver.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/DataDriver.java
@@ -18,23 +18,17 @@
*/
package org.apache.iotdb.db.mpp.execution.driver;
-import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
import org.apache.iotdb.db.engine.storagegroup.IDataRegionForQuery;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.metadata.idtable.IDTable;
-import org.apache.iotdb.db.mpp.execution.exchange.ISinkHandle;
import org.apache.iotdb.db.mpp.execution.operator.Operator;
import org.apache.iotdb.db.mpp.execution.operator.source.DataSourceOperator;
import org.apache.iotdb.db.query.control.FileReaderManager;
-import org.apache.iotdb.tsfile.read.filter.basic.Filter;
import com.google.common.util.concurrent.SettableFuture;
import javax.annotation.concurrent.NotThreadSafe;
-import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
@@ -56,8 +50,8 @@ public class DataDriver extends Driver {
/** unClosed tsfile used in this fragment instance */
private Set<TsFileResource> unClosedFilePaths;
- public DataDriver(Operator root, ISinkHandle sinkHandle, DataDriverContext driverContext) {
- super(root, sinkHandle, driverContext);
+ public DataDriver(Operator root, DriverContext driverContext) {
+ super(root, driverContext);
this.closedFilePaths = new HashSet<>();
this.unClosedFilePaths = new HashSet<>();
}
@@ -69,7 +63,7 @@ public class DataDriver extends Driver {
initialize();
} catch (Throwable t) {
LOGGER.error(
- "Failed to do the initialization for fragment instance {} ", driverContext.getId(), t);
+ "Failed to do the initialization for driver {} ", driverContext.getDriverTaskID(), t);
driverContext.failed(t);
blockedFuture.setException(t);
return false;
@@ -98,7 +92,7 @@ public class DataDriver extends Driver {
* init seq file list and unseq file list in QueryDataSource and set it into each SourceNode TODO
* we should change all the blocked lock operation into tryLock
*/
- private void initialize() throws QueryProcessException {
+ private void initialize() {
long startTime = System.nanoTime();
try {
List<DataSourceOperator> sourceOperators =
@@ -127,32 +121,19 @@ public class DataDriver extends Driver {
* The method is called in mergeLock() when executing query. This method will get all the
* QueryDataSource needed for this query
*/
- private QueryDataSource initQueryDataSource() throws QueryProcessException {
+ private QueryDataSource initQueryDataSource() {
DataDriverContext context = (DataDriverContext) driverContext;
IDataRegionForQuery dataRegion = context.getDataRegion();
dataRegion.readLock();
try {
- List<PartialPath> pathList = new ArrayList<>();
- Set<String> selectedDeviceIdSet = new HashSet<>();
- for (PartialPath path : context.getPaths()) {
- PartialPath translatedPath = IDTable.translateQueryPath(path);
- pathList.add(translatedPath);
- selectedDeviceIdSet.add(translatedPath.getDevice());
- }
- Filter timeFilter = context.getTimeFilter();
- QueryDataSource dataSource =
- dataRegion.query(
- pathList,
- // when all the selected series are under the same device, the QueryDataSource will be
- // filtered according to timeIndex
- selectedDeviceIdSet.size() == 1 ? selectedDeviceIdSet.iterator().next() : null,
- driverContext.getFragmentInstanceContext(),
- timeFilter != null ? timeFilter.copy() : null);
+ QueryDataSource dataSource = ((DataDriverContext) driverContext).getSharedQueryDataSource();
// used files should be added before mergeLock is unlocked, or they may be deleted by
// running merge
- addUsedFilesForQuery(dataSource);
+ if (dataSource != null) {
+ addUsedFilesForQuery(dataSource);
+ }
return dataSource;
} finally {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/DataDriverContext.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/DataDriverContext.java
index dec4cf656c..8dca4dcfe5 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/DataDriverContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/DataDriverContext.java
@@ -19,46 +19,58 @@
package org.apache.iotdb.db.mpp.execution.driver;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
import org.apache.iotdb.db.engine.storagegroup.IDataRegionForQuery;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.operator.source.DataSourceOperator;
-import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import java.util.ArrayList;
import java.util.List;
/** TODO Add javadoc for context */
public class DataDriverContext extends DriverContext {
+
private final List<PartialPath> paths;
- private final Filter timeFilter;
- private final IDataRegionForQuery dataRegion;
private final List<DataSourceOperator> sourceOperators;
- public DataDriverContext(
- FragmentInstanceContext fragmentInstanceContext,
- List<PartialPath> paths,
- Filter timeFilter,
- IDataRegionForQuery dataRegion,
- List<DataSourceOperator> sourceOperators) {
- super(fragmentInstanceContext);
- this.paths = paths;
- this.timeFilter = timeFilter;
- this.dataRegion = dataRegion;
- this.sourceOperators = sourceOperators;
+ public DataDriverContext(FragmentInstanceContext fragmentInstanceContext, int pipelineId) {
+ super(fragmentInstanceContext, pipelineId);
+ this.paths = new ArrayList<>();
+ this.sourceOperators = new ArrayList<>();
+ }
+
+ public DataDriverContext(DataDriverContext parentContext, int pipelineId) {
+ super(parentContext.getFragmentInstanceContext(), pipelineId);
+ this.paths = new ArrayList<>();
+ this.sourceOperators = new ArrayList<>();
+ }
+
+ public void addPath(PartialPath path) {
+ this.paths.add(path);
+ }
+
+ public void addSourceOperator(DataSourceOperator sourceOperator) {
+ this.sourceOperators.add(sourceOperator);
}
public List<PartialPath> getPaths() {
return paths;
}
- public Filter getTimeFilter() {
- return timeFilter;
+ public IDataRegionForQuery getDataRegion() {
+ return getFragmentInstanceContext().getDataRegion();
}
- public IDataRegionForQuery getDataRegion() {
- return dataRegion;
+ public QueryDataSource getSharedQueryDataSource() {
+ return getFragmentInstanceContext().getSharedQueryDataSource();
}
public List<DataSourceOperator> getSourceOperators() {
return sourceOperators;
}
+
+ @Override
+ public DriverContext createSubDriverContext(int pipelineId) {
+ return new DataDriverContext(this, pipelineId);
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/Driver.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/Driver.java
index c5c56452e6..14554bb8fb 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/Driver.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/Driver.java
@@ -18,10 +18,10 @@
*/
package org.apache.iotdb.db.mpp.execution.driver;
-import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.execution.exchange.ISinkHandle;
import org.apache.iotdb.db.mpp.execution.operator.Operator;
import org.apache.iotdb.db.mpp.execution.operator.OperatorContext;
+import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTaskId;
import org.apache.iotdb.db.mpp.metric.QueryMetricsManager;
import org.apache.iotdb.tsfile.read.common.block.TsBlock;
@@ -53,9 +53,9 @@ public abstract class Driver implements IDriver {
protected static final Logger LOGGER = LoggerFactory.getLogger(Driver.class);
+ protected final DriverContext driverContext;
protected final Operator root;
protected final ISinkHandle sinkHandle;
- protected final DriverContext driverContext;
protected final AtomicReference<SettableFuture<?>> driverBlockedFuture = new AtomicReference<>();
protected final AtomicReference<State> state = new AtomicReference<>(State.ALIVE);
@@ -69,12 +69,12 @@ public abstract class Driver implements IDriver {
DESTROYED
}
- protected Driver(Operator root, ISinkHandle sinkHandle, DriverContext driverContext) {
+ protected Driver(Operator root, DriverContext driverContext) {
checkNotNull(root, "root Operator should not be null");
- checkNotNull(sinkHandle, "SinkHandle should not be null");
- this.root = root;
- this.sinkHandle = sinkHandle;
+ checkNotNull(driverContext.getSinkHandle(), "SinkHandle should not be null");
this.driverContext = driverContext;
+ this.root = root;
+ this.sinkHandle = driverContext.getSinkHandle();
// initially the driverBlockedFuture is not blocked (it is completed)
SettableFuture<Void> future = SettableFuture.create();
@@ -146,8 +146,13 @@ public abstract class Driver implements IDriver {
}
@Override
- public FragmentInstanceId getInfo() {
- return driverContext.getId();
+ public DriverTaskId getDriverTaskId() {
+ return driverContext.getDriverTaskID();
+ }
+
+ @Override
+ public void setDriverTaskId(DriverTaskId driverTaskId) {
+ this.driverContext.setDriverTaskID(driverTaskId);
}
@Override
@@ -327,7 +332,10 @@ public abstract class Driver implements IDriver {
// this shouldn't happen but be safe
inFlightException =
addSuppressedException(
- inFlightException, t, "Error destroying driver for task %s", driverContext.getId());
+ inFlightException,
+ t,
+ "Error destroying driver for task %s",
+ driverContext.getDriverTaskID());
} finally {
releaseResource();
}
@@ -350,8 +358,7 @@ public abstract class Driver implements IDriver {
sinkHandle.setNoMoreTsBlocks();
// record operator execution statistics to metrics
- List<OperatorContext> operatorContexts =
- driverContext.getFragmentInstanceContext().getOperatorContexts();
+ List<OperatorContext> operatorContexts = driverContext.getOperatorContexts();
for (OperatorContext operatorContext : operatorContexts) {
String operatorType = operatorContext.getOperatorType();
QUERY_METRICS.recordOperatorExecutionCost(
@@ -368,9 +375,9 @@ public abstract class Driver implements IDriver {
addSuppressedException(
inFlightException,
t,
- "Error closing operator {} for fragment instance {}",
+ "Error closing operator {} for driver task {}",
root.getOperatorContext().getOperatorId(),
- driverContext.getId());
+ driverContext.getDriverTaskID());
} finally {
// reset the interrupted flag
if (wasInterrupted) {
@@ -399,6 +406,7 @@ public abstract class Driver implements IDriver {
}
private static class DriverLock {
+
private final ReentrantLock lock = new ReentrantLock();
@GuardedBy("this")
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/DriverContext.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/DriverContext.java
index 1fa3bacdc2..95a5861978 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/DriverContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/DriverContext.java
@@ -18,23 +18,91 @@
*/
package org.apache.iotdb.db.mpp.execution.driver;
-import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
+import org.apache.iotdb.db.mpp.execution.exchange.ISinkHandle;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
+import org.apache.iotdb.db.mpp.execution.operator.OperatorContext;
+import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTaskId;
+import org.apache.iotdb.db.mpp.execution.timer.RuleBasedTimeSliceAllocator;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
+import java.util.ArrayList;
+import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
+import static com.google.common.base.Preconditions.checkArgument;
+
public class DriverContext {
+ private boolean inputDriver = true;
+ private DriverTaskId driverTaskID;
private final FragmentInstanceContext fragmentInstanceContext;
+ private final List<OperatorContext> operatorContexts = new ArrayList<>();
+ private ISinkHandle sinkHandle;
+ private final RuleBasedTimeSliceAllocator timeSliceAllocator;
private final AtomicBoolean finished = new AtomicBoolean();
- public DriverContext(FragmentInstanceContext fragmentInstanceContext) {
+ public DriverContext(FragmentInstanceContext fragmentInstanceContext, int pipelineId) {
this.fragmentInstanceContext = fragmentInstanceContext;
+ this.driverTaskID = new DriverTaskId(fragmentInstanceContext.getId(), pipelineId);
+ this.timeSliceAllocator = new RuleBasedTimeSliceAllocator();
+ }
+
+ public OperatorContext addOperatorContext(
+ int operatorId, PlanNodeId planNodeId, String operatorType) {
+ checkArgument(operatorId >= 0, "operatorId is negative");
+
+ for (OperatorContext operatorContext : operatorContexts) {
+ checkArgument(
+ operatorId != operatorContext.getOperatorId(),
+ "A context already exists for operatorId %s",
+ operatorId);
+ }
+
+ OperatorContext operatorContext =
+ new OperatorContext(operatorId, planNodeId, operatorType, this);
+ operatorContexts.add(operatorContext);
+ return operatorContext;
+ }
+
+ public DriverContext createSubDriverContext(int pipelineId) {
+ throw new UnsupportedOperationException();
+ }
+
+ public void setSinkHandle(ISinkHandle sinkHandle) {
+ this.sinkHandle = sinkHandle;
+ }
+
+ public ISinkHandle getSinkHandle() {
+ return sinkHandle;
+ }
+
+ public boolean isInputDriver() {
+ return inputDriver;
+ }
+
+ public void setInputDriver(boolean inputDriver) {
+ this.inputDriver = inputDriver;
+ }
+
+ public List<OperatorContext> getOperatorContexts() {
+ return operatorContexts;
+ }
+
+ public RuleBasedTimeSliceAllocator getTimeSliceAllocator() {
+ return timeSliceAllocator;
+ }
+
+ public int getPipelineId() {
+ return driverTaskID.getPipelineId();
+ }
+
+ public DriverTaskId getDriverTaskID() {
+ return driverTaskID;
}
- public FragmentInstanceId getId() {
- return fragmentInstanceContext.getId();
+ public void setDriverTaskID(DriverTaskId driverTaskID) {
+ this.driverTaskID = driverTaskID;
}
public FragmentInstanceContext getFragmentInstanceContext() {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/IDriver.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/IDriver.java
index 0d097bfc7a..8a55d098de 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/IDriver.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/IDriver.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.mpp.execution.driver;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.execution.exchange.ISinkHandle;
+import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTaskId;
import com.google.common.util.concurrent.ListenableFuture;
import io.airlift.units.Duration;
@@ -54,7 +55,9 @@ public interface IDriver {
*
* @return a {@link FragmentInstanceId} instance.
*/
- FragmentInstanceId getInfo();
+ DriverTaskId getDriverTaskId();
+
+ void setDriverTaskId(DriverTaskId driverTaskId);
/** clear resource used by this fragment instance */
void close();
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/SchemaDriver.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/SchemaDriver.java
index 01ed2315b8..19847f83f0 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/SchemaDriver.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/SchemaDriver.java
@@ -18,7 +18,6 @@
*/
package org.apache.iotdb.db.mpp.execution.driver;
-import org.apache.iotdb.db.mpp.execution.exchange.ISinkHandle;
import org.apache.iotdb.db.mpp.execution.operator.Operator;
import com.google.common.util.concurrent.SettableFuture;
@@ -29,8 +28,8 @@ import javax.annotation.concurrent.NotThreadSafe;
@NotThreadSafe
public class SchemaDriver extends Driver {
- public SchemaDriver(Operator root, ISinkHandle sinkHandle, SchemaDriverContext driverContext) {
- super(root, sinkHandle, driverContext);
+ public SchemaDriver(Operator root, SchemaDriverContext driverContext) {
+ super(root, driverContext);
}
@Override
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/SchemaDriverContext.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/SchemaDriverContext.java
index d3ef6aff67..495a2779ee 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/SchemaDriverContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/SchemaDriverContext.java
@@ -28,11 +28,22 @@ public class SchemaDriverContext extends DriverContext {
public SchemaDriverContext(
FragmentInstanceContext fragmentInstanceContext, ISchemaRegion schemaRegion) {
- super(fragmentInstanceContext);
+ // TODO whether schema driver need to be split to pipeline, default 0 now
+ super(fragmentInstanceContext, 0);
this.schemaRegion = schemaRegion;
}
+ public SchemaDriverContext(SchemaDriverContext parentContext, int pipelineId) {
+ super(parentContext.getFragmentInstanceContext(), pipelineId);
+ this.schemaRegion = parentContext.schemaRegion;
+ }
+
public ISchemaRegion getSchemaRegion() {
return schemaRegion;
}
+
+ @Override
+ public DriverContext createSubDriverContext(int pipelineId) {
+ return new SchemaDriverContext(this, pipelineId);
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/IMPPDataExchangeManager.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/IMPPDataExchangeManager.java
index dfc686e699..c02595374c 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/IMPPDataExchangeManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/IMPPDataExchangeManager.java
@@ -44,7 +44,7 @@ public interface IMPPDataExchangeManager {
String localPlanNodeId,
FragmentInstanceContext instanceContext);
- ISinkHandle createLocalSinkHandle(
+ ISinkHandle createLocalSinkHandleForFragment(
TFragmentInstanceId localFragmentInstanceId,
TFragmentInstanceId remoteFragmentInstanceId,
String remotePlanNodeId,
@@ -69,7 +69,7 @@ public interface IMPPDataExchangeManager {
TFragmentInstanceId remoteFragmentInstanceId,
IMPPDataExchangeManagerCallback<Throwable> onFailureCallback);
- ISourceHandle createLocalSourceHandle(
+ ISourceHandle createLocalSourceHandleForFragment(
TFragmentInstanceId localFragmentInstanceId,
String localPlanNodeId,
TFragmentInstanceId remoteFragmentInstanceId,
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/LocalSinkHandle.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/LocalSinkHandle.java
index 575efdb800..0a1d666409 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/LocalSinkHandle.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/LocalSinkHandle.java
@@ -39,9 +39,7 @@ public class LocalSinkHandle implements ISinkHandle {
private static final Logger logger = LoggerFactory.getLogger(LocalSinkHandle.class);
- private final TFragmentInstanceId remoteFragmentInstanceId;
- private final String remotePlanNodeId;
- private final TFragmentInstanceId localFragmentInstanceId;
+ private TFragmentInstanceId localFragmentInstanceId;
private final SinkHandleListener sinkHandleListener;
private final SharedTsBlockQueue queue;
@@ -51,14 +49,17 @@ public class LocalSinkHandle implements ISinkHandle {
private static final QueryMetricsManager QUERY_METRICS = QueryMetricsManager.getInstance();
+ public LocalSinkHandle(SharedTsBlockQueue queue, SinkHandleListener sinkHandleListener) {
+ this.sinkHandleListener = Validate.notNull(sinkHandleListener);
+ this.queue = Validate.notNull(queue);
+ this.queue.setSinkHandle(this);
+ blocked = queue.getCanAddTsBlock();
+ }
+
public LocalSinkHandle(
- TFragmentInstanceId remoteFragmentInstanceId,
- String remotePlanNodeId,
TFragmentInstanceId localFragmentInstanceId,
SharedTsBlockQueue queue,
SinkHandleListener sinkHandleListener) {
- this.remoteFragmentInstanceId = Validate.notNull(remoteFragmentInstanceId);
- this.remotePlanNodeId = Validate.notNull(remotePlanNodeId);
this.localFragmentInstanceId = Validate.notNull(localFragmentInstanceId);
this.sinkHandleListener = Validate.notNull(sinkHandleListener);
this.queue = Validate.notNull(queue);
@@ -189,15 +190,7 @@ public class LocalSinkHandle implements ISinkHandle {
logger.debug("[EndCloseLocalSinkHandle]");
}
- public TFragmentInstanceId getRemoteFragmentInstanceId() {
- return remoteFragmentInstanceId;
- }
-
- public String getRemotePlanNodeId() {
- return remotePlanNodeId;
- }
-
- SharedTsBlockQueue getSharedTsBlockQueue() {
+ public SharedTsBlockQueue getSharedTsBlockQueue() {
return queue;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/LocalSourceHandle.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/LocalSourceHandle.java
index 44571a6eb4..6487a20399 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/LocalSourceHandle.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/LocalSourceHandle.java
@@ -44,9 +44,8 @@ public class LocalSourceHandle implements ISourceHandle {
private static final Logger logger = LoggerFactory.getLogger(LocalSourceHandle.class);
- private final TFragmentInstanceId remoteFragmentInstanceId;
- private final TFragmentInstanceId localFragmentInstanceId;
- private final String localPlanNodeId;
+ private TFragmentInstanceId localFragmentInstanceId;
+ private String localPlanNodeId;
private final SourceHandleListener sourceHandleListener;
private final SharedTsBlockQueue queue;
private boolean aborted = false;
@@ -60,13 +59,21 @@ public class LocalSourceHandle implements ISourceHandle {
private static final TsBlockSerde serde = new TsBlockSerde();
private static final QueryMetricsManager QUERY_METRICS = QueryMetricsManager.getInstance();
+ // For pipeline
+ public LocalSourceHandle(
+ SharedTsBlockQueue queue, SourceHandleListener sourceHandleListener, String threadName) {
+ this.queue = Validate.notNull(queue);
+ this.queue.setSourceHandle(this);
+ this.sourceHandleListener = Validate.notNull(sourceHandleListener);
+ this.threadName = threadName;
+ }
+
+ // For fragment
public LocalSourceHandle(
- TFragmentInstanceId remoteFragmentInstanceId,
TFragmentInstanceId localFragmentInstanceId,
String localPlanNodeId,
SharedTsBlockQueue queue,
SourceHandleListener sourceHandleListener) {
- this.remoteFragmentInstanceId = Validate.notNull(remoteFragmentInstanceId);
this.localFragmentInstanceId = Validate.notNull(localFragmentInstanceId);
this.localPlanNodeId = Validate.notNull(localPlanNodeId);
this.queue = Validate.notNull(queue);
@@ -238,16 +245,14 @@ public class LocalSourceHandle implements ISourceHandle {
}
}
- public TFragmentInstanceId getRemoteFragmentInstanceId() {
- return remoteFragmentInstanceId;
- }
-
- SharedTsBlockQueue getSharedTsBlockQueue() {
+ public SharedTsBlockQueue getSharedTsBlockQueue() {
return queue;
}
@Override
public void setMaxBytesCanReserve(long maxBytesCanReserve) {
- queue.setMaxBytesCanReserve(maxBytesCanReserve);
+ if (maxBytesCanReserve < queue.getMaxBytesCanReserve()) {
+ queue.setMaxBytesCanReserve(maxBytesCanReserve);
+ }
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/MPPDataExchangeManager.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/MPPDataExchangeManager.java
index 9f90ab9075..885447d0c0 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/MPPDataExchangeManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/MPPDataExchangeManager.java
@@ -22,6 +22,7 @@ package org.apache.iotdb.db.mpp.execution.exchange;
import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.commons.client.IClientManager;
import org.apache.iotdb.commons.client.sync.SyncDataNodeMPPDataExchangeServiceClient;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.memory.LocalMemoryManager;
import org.apache.iotdb.db.mpp.metric.QueryMetricsManager;
@@ -273,6 +274,38 @@ public class MPPDataExchangeManager implements IMPPDataExchangeManager {
}
}
+ /**
+ * Listen to the state changes of a source handle of pipeline. Since we register nothing in the
+ * exchangeManager, so we don't need to remove it too.
+ */
+ static class PipelineSourceHandleListenerImpl implements SourceHandleListener {
+
+ private final IMPPDataExchangeManagerCallback<Throwable> onFailureCallback;
+
+ public PipelineSourceHandleListenerImpl(
+ IMPPDataExchangeManagerCallback<Throwable> onFailureCallback) {
+ this.onFailureCallback = onFailureCallback;
+ }
+
+ @Override
+ public void onFinished(ISourceHandle sourceHandle) {
+ logger.debug("[ScHListenerOnFinish]");
+ }
+
+ @Override
+ public void onAborted(ISourceHandle sourceHandle) {
+ logger.debug("[ScHListenerOnAbort]");
+ }
+
+ @Override
+ public void onFailure(ISourceHandle sourceHandle, Throwable t) {
+ logger.warn("Source handle failed due to: ", t);
+ if (onFailureCallback != null) {
+ onFailureCallback.call(t);
+ }
+ }
+ }
+
/** Listen to the state changes of a sink handle. */
class SinkHandleListenerImpl implements SinkHandleListener {
@@ -324,6 +357,48 @@ public class MPPDataExchangeManager implements IMPPDataExchangeManager {
}
}
+ /**
+ * Listen to the state changes of a sink handle of pipeline. And since the finish of pipeline sink
+ * handle doesn't equal the finish of the whole fragment, therefore we don't need to notify
+ * fragment context. But if it's aborted or failed, it can lead to the total fail.
+ */
+ static class PipelineSinkHandleListenerImpl implements SinkHandleListener {
+
+ private final FragmentInstanceContext context;
+ private final IMPPDataExchangeManagerCallback<Throwable> onFailureCallback;
+
+ public PipelineSinkHandleListenerImpl(
+ FragmentInstanceContext context,
+ IMPPDataExchangeManagerCallback<Throwable> onFailureCallback) {
+ this.context = context;
+ this.onFailureCallback = onFailureCallback;
+ }
+
+ @Override
+ public void onFinish(ISinkHandle sinkHandle) {
+ logger.debug("[SkHListenerOnFinish]");
+ }
+
+ @Override
+ public void onEndOfBlocks(ISinkHandle sinkHandle) {
+ logger.debug("[SkHListenerOnEndOfTsBlocks]");
+ }
+
+ @Override
+ public Optional<Throwable> onAborted(ISinkHandle sinkHandle) {
+ logger.debug("[SkHListenerOnAbort]");
+ return context.getFailureCause();
+ }
+
+ @Override
+ public void onFailure(ISinkHandle sinkHandle, Throwable t) {
+ logger.warn("Sink handle failed due to", t);
+ if (onFailureCallback != null) {
+ onFailureCallback.call(t);
+ }
+ }
+ }
+
private final LocalMemoryManager localMemoryManager;
private final Supplier<TsBlockSerde> tsBlockSerdeFactory;
private final ExecutorService executorService;
@@ -357,7 +432,7 @@ public class MPPDataExchangeManager implements IMPPDataExchangeManager {
}
@Override
- public synchronized ISinkHandle createLocalSinkHandle(
+ public synchronized ISinkHandle createLocalSinkHandleForFragment(
TFragmentInstanceId localFragmentInstanceId,
TFragmentInstanceId remoteFragmentInstanceId,
String remotePlanNodeId,
@@ -390,8 +465,6 @@ public class MPPDataExchangeManager implements IMPPDataExchangeManager {
LocalSinkHandle localSinkHandle =
new LocalSinkHandle(
- remoteFragmentInstanceId,
- remotePlanNodeId,
localFragmentInstanceId,
queue,
new SinkHandleListenerImpl(instanceContext, instanceContext::failed));
@@ -399,6 +472,24 @@ public class MPPDataExchangeManager implements IMPPDataExchangeManager {
return localSinkHandle;
}
+ /**
+ * As we know the upstream and downstream node of shared queue, we don't need to put it into the
+ * sinkHandle map.
+ */
+ public ISinkHandle createLocalSinkHandleForPipeline(
+ DriverContext driverContext, String planNodeId) {
+ logger.debug("Create local sink handle for {}", driverContext.getDriverTaskID());
+ SharedTsBlockQueue queue =
+ new SharedTsBlockQueue(
+ driverContext.getDriverTaskID().getFragmentInstanceId().toThrift(),
+ planNodeId,
+ localMemoryManager);
+ return new LocalSinkHandle(
+ queue,
+ new PipelineSinkHandleListenerImpl(
+ driverContext.getFragmentInstanceContext(), driverContext::failed));
+ }
+
@Override
public ISinkHandle createSinkHandle(
TFragmentInstanceId localFragmentInstanceId,
@@ -435,8 +526,21 @@ public class MPPDataExchangeManager implements IMPPDataExchangeManager {
return sinkHandle;
}
+ /**
+ * As we know the upstream and downstream node of shared queue, we don't need to put it into the
+ * sourceHandle map.
+ */
+ public ISourceHandle createLocalSourceHandleForPipeline(
+ SharedTsBlockQueue queue, DriverContext context) {
+ logger.debug("Create local source handle for {}", context.getDriverTaskID());
+ return new LocalSourceHandle(
+ queue,
+ new PipelineSourceHandleListenerImpl(context::failed),
+ context.getDriverTaskID().toString());
+ }
+
@Override
- public synchronized ISourceHandle createLocalSourceHandle(
+ public synchronized ISourceHandle createLocalSourceHandleForFragment(
TFragmentInstanceId localFragmentInstanceId,
String localPlanNodeId,
TFragmentInstanceId remoteFragmentInstanceId,
@@ -466,7 +570,6 @@ public class MPPDataExchangeManager implements IMPPDataExchangeManager {
}
LocalSourceHandle localSourceHandle =
new LocalSourceHandle(
- remoteFragmentInstanceId,
localFragmentInstanceId,
localPlanNodeId,
queue,
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SharedTsBlockQueue.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SharedTsBlockQueue.java
index 95e64b1828..ade913aebc 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SharedTsBlockQueue.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SharedTsBlockQueue.java
@@ -48,6 +48,7 @@ public class SharedTsBlockQueue {
private final TFragmentInstanceId localFragmentInstanceId;
private final String localPlanNodeId;
+
private final LocalMemoryManager localMemoryManager;
private boolean noMoreTsBlocks = false;
@@ -101,6 +102,10 @@ public class SharedTsBlockQueue {
this.maxBytesCanReserve = maxBytesCanReserve;
}
+ public long getMaxBytesCanReserve() {
+ return maxBytesCanReserve;
+ }
+
public ListenableFuture<Void> isBlocked() {
if (!canAddTsBlock.isDone()) {
canAddTsBlock.set(null);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SinkHandle.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SinkHandle.java
index dc4d1d2803..e2494af0fb 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SinkHandle.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SinkHandle.java
@@ -350,7 +350,7 @@ public class SinkHandle implements ISinkHandle {
@Override
public void setMaxBytesCanReserve(long maxBytesCanReserve) {
- this.maxBytesCanReserve = maxBytesCanReserve;
+ this.maxBytesCanReserve = Math.min(this.maxBytesCanReserve, maxBytesCanReserve);
}
@Override
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SourceHandle.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SourceHandle.java
index 85a9defe30..2e1a4b8a6d 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SourceHandle.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SourceHandle.java
@@ -393,7 +393,7 @@ public class SourceHandle implements ISourceHandle {
@Override
public void setMaxBytesCanReserve(long maxBytesCanReserve) {
- this.maxBytesCanReserve = maxBytesCanReserve;
+ this.maxBytesCanReserve = Math.min(this.maxBytesCanReserve, maxBytesCanReserve);
}
@Override
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/fragment/FragmentInstanceContext.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/fragment/FragmentInstanceContext.java
index 5ddad69257..b1921ba307 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/fragment/FragmentInstanceContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/fragment/FragmentInstanceContext.java
@@ -18,42 +18,43 @@
*/
package org.apache.iotdb.db.mpp.execution.fragment;
+import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.engine.storagegroup.IDataRegionForQuery;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.idtable.IDTable;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.SessionInfo;
-import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
-import org.apache.iotdb.db.mpp.execution.operator.OperatorContext;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.time.ZoneId;
import java.util.ArrayList;
+import java.util.HashSet;
import java.util.List;
import java.util.Optional;
-import java.util.concurrent.ExecutorService;
+import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
-import static com.google.common.base.Preconditions.checkArgument;
-
public class FragmentInstanceContext extends QueryContext {
private static final Logger LOGGER = LoggerFactory.getLogger(FragmentInstanceContext.class);
private static final long END_TIME_INITIAL_VALUE = -1L;
private final FragmentInstanceId id;
- // TODO if we split one fragment instance into multiple pipelines to run, we need to replace it
- // with CopyOnWriteArrayList or some other thread safe data structure
- private final List<OperatorContext> operatorContexts = new ArrayList<>();
-
- private DriverContext driverContext;
-
private final FragmentInstanceStateMachine stateMachine;
+ private IDataRegionForQuery dataRegion;
+ private Filter timeFilter;
+ // Shared by all scan operators in this fragment instance to avoid memory problem
+ private QueryDataSource sharedQueryDataSource;
+
private final long createNanos = System.nanoTime();
private final AtomicLong startNanos = new AtomicLong();
@@ -66,8 +67,6 @@ public class FragmentInstanceContext extends QueryContext {
// session info
private SessionInfo sessionInfo;
- private ExecutorService intoOperationExecutor;
-
// private final GcMonitor gcMonitor;
// private final AtomicLong startNanos = new AtomicLong();
// private final AtomicLong startFullGcCount = new AtomicLong(-1);
@@ -76,13 +75,23 @@ public class FragmentInstanceContext extends QueryContext {
// private final AtomicLong endFullGcCount = new AtomicLong(-1);
// private final AtomicLong endFullGcTimeNanos = new AtomicLong(-1);
+ public static FragmentInstanceContext createFragmentInstanceContext(
+ FragmentInstanceId id, FragmentInstanceStateMachine stateMachine, SessionInfo sessionInfo) {
+ FragmentInstanceContext instanceContext =
+ new FragmentInstanceContext(id, stateMachine, sessionInfo);
+ instanceContext.initialize();
+ instanceContext.start();
+ return instanceContext;
+ }
+
public static FragmentInstanceContext createFragmentInstanceContext(
FragmentInstanceId id,
FragmentInstanceStateMachine stateMachine,
SessionInfo sessionInfo,
- ExecutorService intoOperationExecutor) {
+ IDataRegionForQuery dataRegion,
+ Filter timeFilter) {
FragmentInstanceContext instanceContext =
- new FragmentInstanceContext(id, stateMachine, sessionInfo, intoOperationExecutor);
+ new FragmentInstanceContext(id, stateMachine, sessionInfo, dataRegion, timeFilter);
instanceContext.initialize();
instanceContext.start();
return instanceContext;
@@ -96,12 +105,22 @@ public class FragmentInstanceContext extends QueryContext {
FragmentInstanceId id,
FragmentInstanceStateMachine stateMachine,
SessionInfo sessionInfo,
- ExecutorService intoOperationExecutor) {
+ IDataRegionForQuery dataRegion,
+ Filter timeFilter) {
+ this.id = id;
+ this.stateMachine = stateMachine;
+ this.executionEndTime.set(END_TIME_INITIAL_VALUE);
+ this.sessionInfo = sessionInfo;
+ this.dataRegion = dataRegion;
+ this.timeFilter = timeFilter;
+ }
+
+ private FragmentInstanceContext(
+ FragmentInstanceId id, FragmentInstanceStateMachine stateMachine, SessionInfo sessionInfo) {
this.id = id;
this.stateMachine = stateMachine;
this.executionEndTime.set(END_TIME_INITIAL_VALUE);
this.sessionInfo = sessionInfo;
- this.intoOperationExecutor = intoOperationExecutor;
}
@TestOnly
@@ -109,12 +128,17 @@ public class FragmentInstanceContext extends QueryContext {
FragmentInstanceId id, FragmentInstanceStateMachine stateMachine) {
FragmentInstanceContext instanceContext =
new FragmentInstanceContext(
- id, stateMachine, new SessionInfo(1, "test", ZoneId.systemDefault().getId()), null);
+ id, stateMachine, new SessionInfo(1, "test", ZoneId.systemDefault().getId()));
instanceContext.initialize();
instanceContext.start();
return instanceContext;
}
+ @TestOnly
+ public void setDataRegion(IDataRegionForQuery dataRegion) {
+ this.dataRegion = dataRegion;
+ }
+
// used for compaction
private FragmentInstanceContext(long queryId) {
this.queryId = queryId;
@@ -156,39 +180,10 @@ public class FragmentInstanceContext extends QueryContext {
}
}
- public OperatorContext addOperatorContext(
- int operatorId, PlanNodeId planNodeId, String operatorType) {
- checkArgument(operatorId >= 0, "operatorId is negative");
-
- for (OperatorContext operatorContext : operatorContexts) {
- checkArgument(
- operatorId != operatorContext.getOperatorId(),
- "A context already exists for operatorId %s",
- operatorId);
- }
-
- OperatorContext operatorContext =
- new OperatorContext(operatorId, planNodeId, operatorType, this);
- operatorContexts.add(operatorContext);
- return operatorContext;
- }
-
- public List<OperatorContext> getOperatorContexts() {
- return operatorContexts;
- }
-
public FragmentInstanceId getId() {
return id;
}
- public DriverContext getDriverContext() {
- return driverContext;
- }
-
- public void setDriverContext(DriverContext driverContext) {
- this.driverContext = driverContext;
- }
-
public void failed(Throwable cause) {
stateMachine.failed(cause);
}
@@ -250,7 +245,39 @@ public class FragmentInstanceContext extends QueryContext {
return Optional.ofNullable(stateMachine.getFailureCauses().peek());
}
- public ExecutorService getIntoOperationExecutor() {
- return intoOperationExecutor;
+ public Filter getTimeFilter() {
+ return timeFilter;
+ }
+
+ public IDataRegionForQuery getDataRegion() {
+ return dataRegion;
+ }
+
+ public void initQueryDataSource(List<PartialPath> sourcePaths) throws QueryProcessException {
+ dataRegion.readLock();
+ try {
+ List<PartialPath> pathList = new ArrayList<>();
+ Set<String> selectedDeviceIdSet = new HashSet<>();
+ for (PartialPath path : sourcePaths) {
+ PartialPath translatedPath = IDTable.translateQueryPath(path);
+ pathList.add(translatedPath);
+ selectedDeviceIdSet.add(translatedPath.getDevice());
+ }
+
+ this.sharedQueryDataSource =
+ dataRegion.query(
+ pathList,
+ // when all the selected series are under the same device, the QueryDataSource will be
+ // filtered according to timeIndex
+ selectedDeviceIdSet.size() == 1 ? selectedDeviceIdSet.iterator().next() : null,
+ this,
+ timeFilter != null ? timeFilter.copy() : null);
+ } finally {
+ dataRegion.readUnlock();
+ }
+ }
+
+ public QueryDataSource getSharedQueryDataSource() {
+ return sharedQueryDataSource;
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/fragment/FragmentInstanceExecution.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/fragment/FragmentInstanceExecution.java
index a71ebd0dab..14ada6d7c0 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/fragment/FragmentInstanceExecution.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/fragment/FragmentInstanceExecution.java
@@ -24,11 +24,12 @@ import org.apache.iotdb.db.mpp.execution.exchange.ISinkHandle;
import org.apache.iotdb.db.mpp.execution.schedule.IDriverScheduler;
import org.apache.iotdb.db.utils.SetThreadName;
-import com.google.common.collect.ImmutableList;
import io.airlift.stats.CounterStat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.util.List;
+
import static java.util.Objects.requireNonNull;
import static org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceState.FAILED;
@@ -39,7 +40,7 @@ public class FragmentInstanceExecution {
private final FragmentInstanceContext context;
// it will be set to null while this FI is FINISHED
- private IDriver driver;
+ private List<IDriver> drivers;
// it will be set to null while this FI is FINISHED
private ISinkHandle sinkHandle;
@@ -52,27 +53,29 @@ public class FragmentInstanceExecution {
IDriverScheduler scheduler,
FragmentInstanceId instanceId,
FragmentInstanceContext context,
- IDriver driver,
+ List<IDriver> drivers,
+ ISinkHandle sinkHandle,
FragmentInstanceStateMachine stateMachine,
CounterStat failedInstances,
long timeOut) {
FragmentInstanceExecution execution =
- new FragmentInstanceExecution(instanceId, context, driver, stateMachine);
+ new FragmentInstanceExecution(instanceId, context, drivers, sinkHandle, stateMachine);
execution.initialize(failedInstances, scheduler);
LOGGER.debug("timeout is {}ms.", timeOut);
- scheduler.submitDrivers(instanceId.getQueryId(), ImmutableList.of(driver), timeOut);
+ scheduler.submitDrivers(instanceId.getQueryId(), drivers, timeOut);
return execution;
}
private FragmentInstanceExecution(
FragmentInstanceId instanceId,
FragmentInstanceContext context,
- IDriver driver,
+ List<IDriver> drivers,
+ ISinkHandle sinkHandle,
FragmentInstanceStateMachine stateMachine) {
this.instanceId = instanceId;
this.context = context;
- this.driver = driver;
- this.sinkHandle = driver.getSinkHandle();
+ this.drivers = drivers;
+ this.sinkHandle = sinkHandle;
this.stateMachine = stateMachine;
}
@@ -124,9 +127,11 @@ public class FragmentInstanceExecution {
sinkHandle = null;
// close the driver after sinkHandle is aborted or closed because in driver.close() it
// will try to call ISinkHandle.setNoMoreTsBlocks()
- driver.close();
+ for (IDriver driver : drivers) {
+ driver.close();
+ }
// help for gc
- driver = null;
+ drivers = null;
if (newState.isFailed()) {
scheduler.abortFragmentInstance(instanceId);
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/fragment/FragmentInstanceManager.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/fragment/FragmentInstanceManager.java
index 42a133ecb9..efa8db0bf1 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/fragment/FragmentInstanceManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/fragment/FragmentInstanceManager.java
@@ -24,12 +24,14 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.storagegroup.IDataRegionForQuery;
import org.apache.iotdb.db.metadata.schemaregion.ISchemaRegion;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
-import org.apache.iotdb.db.mpp.execution.driver.DataDriver;
+import org.apache.iotdb.db.mpp.execution.driver.IDriver;
import org.apache.iotdb.db.mpp.execution.driver.SchemaDriver;
+import org.apache.iotdb.db.mpp.execution.exchange.ISinkHandle;
import org.apache.iotdb.db.mpp.execution.schedule.DriverScheduler;
import org.apache.iotdb.db.mpp.execution.schedule.IDriverScheduler;
import org.apache.iotdb.db.mpp.metric.QueryMetricsManager;
import org.apache.iotdb.db.mpp.plan.planner.LocalExecutionPlanner;
+import org.apache.iotdb.db.mpp.plan.planner.PipelineDriverFactory;
import org.apache.iotdb.db.mpp.plan.planner.plan.FragmentInstance;
import org.apache.iotdb.db.utils.SetThreadName;
@@ -38,6 +40,9 @@ import io.airlift.units.Duration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
@@ -123,21 +128,27 @@ public class FragmentInstanceManager {
fragmentInstanceId,
stateMachine,
instance.getSessionInfo(),
- intoOperationExecutor));
+ dataRegion,
+ instance.getTimeFilter()));
try {
- DataDriver driver =
+ List<PipelineDriverFactory> driverFactories =
planner.plan(
instance.getFragment().getPlanNodeTree(),
instance.getFragment().getTypeProvider(),
- context,
- instance.getTimeFilter(),
- dataRegion);
+ context);
+
+ List<IDriver> drivers = new ArrayList<>();
+ driverFactories.forEach(factory -> drivers.add(factory.createDriver()));
+ // get the sinkHandle of last driver
+ ISinkHandle sinkHandle = drivers.get(drivers.size() - 1).getSinkHandle();
+
return createFragmentInstanceExecution(
scheduler,
instanceId,
context,
- driver,
+ drivers,
+ sinkHandle,
stateMachine,
failedInstances,
instance.getTimeOut());
@@ -181,10 +192,7 @@ public class FragmentInstanceManager {
instanceId,
fragmentInstanceId ->
createFragmentInstanceContext(
- fragmentInstanceId,
- stateMachine,
- instance.getSessionInfo(),
- intoOperationExecutor));
+ fragmentInstanceId, stateMachine, instance.getSessionInfo()));
try {
SchemaDriver driver =
@@ -193,7 +201,8 @@ public class FragmentInstanceManager {
scheduler,
instanceId,
context,
- driver,
+ Collections.singletonList(driver),
+ driver.getSinkHandle(),
stateMachine,
failedInstances,
instance.getTimeOut());
@@ -294,6 +303,10 @@ public class FragmentInstanceManager {
.forEach(entry -> entry.getValue().failed(new TimeoutException()));
}
+ public ExecutorService getIntoOperationExecutor() {
+ return intoOperationExecutor;
+ }
+
private static class InstanceHolder {
private InstanceHolder() {}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/OperatorContext.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/OperatorContext.java
index 3e9620a1b2..c5c0db783d 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/OperatorContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/OperatorContext.java
@@ -18,7 +18,9 @@
*/
package org.apache.iotdb.db.mpp.execution.operator;
+import org.apache.iotdb.commons.utils.TestOnly;
import org.apache.iotdb.db.mpp.common.SessionInfo;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
@@ -34,24 +36,33 @@ import java.util.Objects;
public class OperatorContext {
private final int operatorId;
+ // It seems it's never used.
private final PlanNodeId planNodeId;
private final String operatorType;
- private final FragmentInstanceContext instanceContext;
-
+ private DriverContext driverContext;
private Duration maxRunTime;
private long totalExecutionTimeInNanos = 0L;
private long nextCalledCount = 0L;
+ public OperatorContext(
+ int operatorId, PlanNodeId planNodeId, String operatorType, DriverContext driverContext) {
+ this.operatorId = operatorId;
+ this.planNodeId = planNodeId;
+ this.operatorType = operatorType;
+ this.driverContext = driverContext;
+ }
+
+ @TestOnly
public OperatorContext(
int operatorId,
PlanNodeId planNodeId,
String operatorType,
- FragmentInstanceContext instanceContext) {
+ FragmentInstanceContext fragmentInstanceContext) {
this.operatorId = operatorId;
this.planNodeId = planNodeId;
this.operatorType = operatorType;
- this.instanceContext = instanceContext;
+ this.driverContext = new DriverContext(fragmentInstanceContext, 0);
}
public int getOperatorId() {
@@ -62,8 +73,17 @@ public class OperatorContext {
return operatorType;
}
+ public DriverContext getDriverContext() {
+ return driverContext;
+ }
+
+ public void setDriverContext(DriverContext driverContext) {
+ this.driverContext = driverContext;
+ }
+
+ // TODO forbid get instance context from operator directly
public FragmentInstanceContext getInstanceContext() {
- return instanceContext;
+ return driverContext.getFragmentInstanceContext();
}
public Duration getMaxRunTime() {
@@ -75,7 +95,7 @@ public class OperatorContext {
}
public SessionInfo getSessionInfo() {
- return instanceContext.getSessionInfo();
+ return getInstanceContext().getSessionInfo();
}
public void recordExecutionTime(long executionTimeInNanos) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/SchemaDriverContext.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/factory/OperatorFactory.java
similarity index 54%
copy from server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/SchemaDriverContext.java
copy to server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/factory/OperatorFactory.java
index d3ef6aff67..91d77d8e02 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/SchemaDriverContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/factory/OperatorFactory.java
@@ -7,7 +7,7 @@
* "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
+ * 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
@@ -16,23 +16,23 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.db.mpp.execution.driver;
-import org.apache.iotdb.db.metadata.schemaregion.ISchemaRegion;
-import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
+package org.apache.iotdb.db.mpp.execution.operator.factory;
-/** TODO Add javadoc for context */
-public class SchemaDriverContext extends DriverContext {
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
+import org.apache.iotdb.db.mpp.execution.operator.Operator;
- private final ISchemaRegion schemaRegion;
+public interface OperatorFactory {
+ Operator createOperator(DriverContext driverContext);
- public SchemaDriverContext(
- FragmentInstanceContext fragmentInstanceContext, ISchemaRegion schemaRegion) {
- super(fragmentInstanceContext);
- this.schemaRegion = schemaRegion;
- }
+ /**
+ * Declare that createOperator will not be called any more and release any resources associated
+ * with this factory.
+ *
+ * <p>This method will be called only once. Implementation doesn't need to worry about duplicate
+ * invocations.
+ */
+ void noMoreOperators();
- public ISchemaRegion getSchemaRegion() {
- return schemaRegion;
- }
+ OperatorFactory duplicate();
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/SchemaDriverContext.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/factory/SourceOperatorFactory.java
similarity index 54%
copy from server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/SchemaDriverContext.java
copy to server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/factory/SourceOperatorFactory.java
index d3ef6aff67..69479f46a3 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/driver/SchemaDriverContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/factory/SourceOperatorFactory.java
@@ -7,7 +7,7 @@
* "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
+ * 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
@@ -16,23 +16,21 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.db.mpp.execution.driver;
-import org.apache.iotdb.db.metadata.schemaregion.ISchemaRegion;
-import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
+package org.apache.iotdb.db.mpp.execution.operator.factory;
-/** TODO Add javadoc for context */
-public class SchemaDriverContext extends DriverContext {
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
+import org.apache.iotdb.db.mpp.execution.operator.source.SourceOperator;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
- private final ISchemaRegion schemaRegion;
+public interface SourceOperatorFactory extends OperatorFactory {
+ PlanNodeId getSourceId();
- public SchemaDriverContext(
- FragmentInstanceContext fragmentInstanceContext, ISchemaRegion schemaRegion) {
- super(fragmentInstanceContext);
- this.schemaRegion = schemaRegion;
- }
+ @Override
+ SourceOperator createOperator(DriverContext driverContext);
- public ISchemaRegion getSchemaRegion() {
- return schemaRegion;
+ @Override
+ default OperatorFactory duplicate() {
+ throw new UnsupportedOperationException("Source operator factories cannot be duplicated");
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/join/TimeJoinOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/join/TimeJoinOperator.java
index 470a52cc08..74bbada77c 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/join/TimeJoinOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/join/TimeJoinOperator.java
@@ -194,7 +194,6 @@ public class TimeJoinOperator extends AbstractProcessOperator {
System.arraycopy(shadowInputIndex, 0, inputIndex, 0, inputOperatorsCount);
resultTsBlock = tsBlockBuilder.build();
-
return checkTsBlockSizeAndGetResult();
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/last/AbstractUpdateLastCacheOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/last/AbstractUpdateLastCacheOperator.java
index 9d02bd91dc..60095e9365 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/last/AbstractUpdateLastCacheOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/last/AbstractUpdateLastCacheOperator.java
@@ -73,7 +73,7 @@ public abstract class AbstractUpdateLastCacheOperator implements ProcessOperator
protected String getDatabaseName() {
if (databaseName == null) {
databaseName =
- ((DataDriverContext) operatorContext.getInstanceContext().getDriverContext())
+ ((DataDriverContext) operatorContext.getDriverContext())
.getDataRegion()
.getDatabaseName();
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/CountGroupByLevelScanOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/CountGroupByLevelScanOperator.java
index 30671257cb..e1d4152ef2 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/CountGroupByLevelScanOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/CountGroupByLevelScanOperator.java
@@ -97,8 +97,7 @@ public class CountGroupByLevelScanOperator<T extends ISchemaInfo> implements Sou
public ISchemaReader<T> createTimeSeriesReader() {
return schemaSource.getSchemaReader(
- ((SchemaDriverContext) operatorContext.getInstanceContext().getDriverContext())
- .getSchemaRegion());
+ ((SchemaDriverContext) operatorContext.getDriverContext()).getSchemaRegion());
}
private TsBlock generateResult() {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaQueryScanOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/NodePathsSchemaScanOperator.java
similarity index 53%
copy from server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaQueryScanOperator.java
copy to server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/NodePathsSchemaScanOperator.java
index 886deb523a..6902648e44 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaQueryScanOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/NodePathsSchemaScanOperator.java
@@ -16,111 +16,62 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.mpp.execution.operator.schema;
+import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.metadata.query.info.ISchemaInfo;
+import org.apache.iotdb.db.metadata.plan.schemaregion.impl.read.SchemaRegionReadPlanFactory;
+import org.apache.iotdb.db.metadata.plan.schemaregion.read.IShowNodesPlan;
+import org.apache.iotdb.db.metadata.query.info.INodeSchemaInfo;
import org.apache.iotdb.db.metadata.query.reader.ISchemaReader;
import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
+import org.apache.iotdb.db.mpp.common.header.ColumnHeaderConstant;
import org.apache.iotdb.db.mpp.execution.driver.SchemaDriverContext;
import org.apache.iotdb.db.mpp.execution.operator.OperatorContext;
-import org.apache.iotdb.db.mpp.execution.operator.schema.source.ISchemaSource;
import org.apache.iotdb.db.mpp.execution.operator.source.SourceOperator;
import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.common.block.TsBlock;
import org.apache.iotdb.tsfile.read.common.block.TsBlockBuilder;
+import org.apache.iotdb.tsfile.utils.Binary;
import java.util.List;
-import java.util.NoSuchElementException;
import java.util.stream.Collectors;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
import static org.apache.iotdb.tsfile.read.common.block.TsBlockBuilderStatus.DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES;
-public class SchemaQueryScanOperator<T extends ISchemaInfo> implements SourceOperator {
+public class NodePathsSchemaScanOperator implements SourceOperator {
private static final long MAX_SIZE = DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES;
- protected PlanNodeId sourceId;
+ private final PlanNodeId sourceId;
- protected OperatorContext operatorContext;
+ private final OperatorContext operatorContext;
- private final ISchemaSource<T> schemaSource;
+ private final PartialPath partialPath;
- protected int limit;
- protected int offset;
- protected PartialPath partialPath;
- protected boolean isPrefixPath;
+ private final int level;
- private String database;
+ private boolean isFinished;
private final List<TSDataType> outputDataTypes;
- private ISchemaReader<T> schemaReader;
+ private ISchemaReader<INodeSchemaInfo> nodeReader;
- protected SchemaQueryScanOperator(
- PlanNodeId sourceId,
- OperatorContext operatorContext,
- int limit,
- int offset,
- PartialPath partialPath,
- boolean isPrefixPath,
- List<TSDataType> outputDataTypes) {
- this.operatorContext = operatorContext;
- this.limit = limit;
- this.offset = offset;
- this.partialPath = partialPath;
- this.isPrefixPath = isPrefixPath;
- this.sourceId = sourceId;
- this.outputDataTypes = outputDataTypes;
- this.schemaSource = null;
- }
-
- public SchemaQueryScanOperator(
- PlanNodeId sourceId, OperatorContext operatorContext, ISchemaSource<T> schemaSource) {
+ public NodePathsSchemaScanOperator(
+ PlanNodeId sourceId, OperatorContext operatorContext, PartialPath partialPath, int level) {
this.sourceId = sourceId;
this.operatorContext = operatorContext;
- this.schemaSource = schemaSource;
+ this.partialPath = partialPath;
+ this.level = level;
this.outputDataTypes =
- schemaSource.getInfoQueryColumnHeaders().stream()
+ ColumnHeaderConstant.showChildPathsColumnHeaders.stream()
.map(ColumnHeader::getColumnType)
.collect(Collectors.toList());
}
- protected ISchemaReader<T> createSchemaReader() {
- return schemaSource.getSchemaReader(
- ((SchemaDriverContext) operatorContext.getInstanceContext().getDriverContext())
- .getSchemaRegion());
- }
-
- protected void setColumns(T element, TsBlockBuilder builder) {
- schemaSource.transformToTsBlockColumns(element, builder, getDatabase());
- }
-
- public PartialPath getPartialPath() {
- return partialPath;
- }
-
- public int getLimit() {
- return limit;
- }
-
- public int getOffset() {
- return offset;
- }
-
- public void setLimit(int limit) {
- this.limit = limit;
- }
-
- public void setOffset(int offset) {
- this.offset = offset;
- }
-
- public boolean isPrefixPath() {
- return isPrefixPath;
- }
-
@Override
public OperatorContext getOperatorContext() {
return operatorContext;
@@ -128,14 +79,17 @@ public class SchemaQueryScanOperator<T extends ISchemaInfo> implements SourceOpe
@Override
public TsBlock next() {
- if (!hasNext()) {
- throw new NoSuchElementException();
- }
+ isFinished = true;
TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(outputDataTypes);
- T element;
- while (schemaReader.hasNext()) {
- element = schemaReader.next();
- setColumns(element, tsBlockBuilder);
+ INodeSchemaInfo nodeSchemaInfo;
+ while (nodeReader.hasNext()) {
+ nodeSchemaInfo = nodeReader.next();
+ tsBlockBuilder.getTimeColumnBuilder().writeLong(0L);
+ tsBlockBuilder.getColumnBuilder(0).writeBinary(new Binary(nodeSchemaInfo.getFullPath()));
+ tsBlockBuilder
+ .getColumnBuilder(1)
+ .writeBinary(new Binary(String.valueOf(nodeSchemaInfo.getNodeType().getNodeType())));
+ tsBlockBuilder.declarePosition();
if (tsBlockBuilder.getRetainedSizeInBytes() >= MAX_SIZE) {
break;
}
@@ -145,10 +99,29 @@ public class SchemaQueryScanOperator<T extends ISchemaInfo> implements SourceOpe
@Override
public boolean hasNext() {
- if (schemaReader == null) {
- schemaReader = createSchemaReader();
+ if (nodeReader == null) {
+ nodeReader = createReader();
+ }
+ return nodeReader.hasNext();
+ }
+
+ private ISchemaReader<INodeSchemaInfo> createReader() {
+ IShowNodesPlan showNodesPlan;
+ if (-1 == level) {
+ showNodesPlan =
+ SchemaRegionReadPlanFactory.getShowNodesPlan(
+ partialPath.concatNode(ONE_LEVEL_PATH_WILDCARD));
+ } else {
+ showNodesPlan = SchemaRegionReadPlanFactory.getShowNodesPlan(partialPath, level, false);
+ }
+ try {
+ return nodeReader =
+ ((SchemaDriverContext) operatorContext.getDriverContext())
+ .getSchemaRegion()
+ .getNodeReader(showNodesPlan);
+ } catch (MetadataException e) {
+ throw new RuntimeException(e.getMessage(), e);
}
- return schemaReader.hasNext();
}
@Override
@@ -176,21 +149,11 @@ public class SchemaQueryScanOperator<T extends ISchemaInfo> implements SourceOpe
return 0L;
}
- protected String getDatabase() {
- if (database == null) {
- database =
- ((SchemaDriverContext) operatorContext.getInstanceContext().getDriverContext())
- .getSchemaRegion()
- .getStorageGroupFullPath();
- }
- return database;
- }
-
@Override
public void close() throws Exception {
- if (schemaReader != null) {
- schemaReader.close();
- schemaReader = null;
+ if (nodeReader != null) {
+ nodeReader.close();
+ nodeReader = null;
}
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaCountOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaCountOperator.java
index c406be3745..7de9660365 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaCountOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaCountOperator.java
@@ -56,8 +56,7 @@ public class SchemaCountOperator<T extends ISchemaInfo> implements SourceOperato
}
private final ISchemaRegion getSchemaRegion() {
- return ((SchemaDriverContext) operatorContext.getInstanceContext().getDriverContext())
- .getSchemaRegion();
+ return ((SchemaDriverContext) operatorContext.getDriverContext()).getSchemaRegion();
}
private ISchemaReader<T> createSchemaReader() {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaQueryScanOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaQueryScanOperator.java
index 886deb523a..ab05353858 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaQueryScanOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaQueryScanOperator.java
@@ -89,8 +89,7 @@ public class SchemaQueryScanOperator<T extends ISchemaInfo> implements SourceOpe
protected ISchemaReader<T> createSchemaReader() {
return schemaSource.getSchemaReader(
- ((SchemaDriverContext) operatorContext.getInstanceContext().getDriverContext())
- .getSchemaRegion());
+ ((SchemaDriverContext) operatorContext.getDriverContext()).getSchemaRegion());
}
protected void setColumns(T element, TsBlockBuilder builder) {
@@ -179,7 +178,7 @@ public class SchemaQueryScanOperator<T extends ISchemaInfo> implements SourceOpe
protected String getDatabase() {
if (database == null) {
database =
- ((SchemaDriverContext) operatorContext.getInstanceContext().getDriverContext())
+ ((SchemaDriverContext) operatorContext.getDriverContext())
.getSchemaRegion()
.getStorageGroupFullPath();
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AlignedSeriesScanUtil.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AlignedSeriesScanUtil.java
index 52e514eee4..79a494b35a 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AlignedSeriesScanUtil.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AlignedSeriesScanUtil.java
@@ -21,7 +21,6 @@ package org.apache.iotdb.db.mpp.execution.operator.source;
import org.apache.iotdb.commons.path.AlignedPath;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.query.context.QueryContext;
import org.apache.iotdb.db.query.reader.universal.AlignedDescPriorityMergeReader;
import org.apache.iotdb.db.query.reader.universal.AlignedPriorityMergeReader;
@@ -49,7 +48,7 @@ public class AlignedSeriesScanUtil extends SeriesScanUtil {
public AlignedSeriesScanUtil(
PartialPath seriesPath,
Set<String> allSensors,
- FragmentInstanceContext context,
+ QueryContext context,
Filter timeFilter,
Filter valueFilter,
boolean ascending) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/ExchangeOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/ExchangeOperator.java
index 110079809b..7e7498fbc1 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/ExchangeOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/ExchangeOperator.java
@@ -84,6 +84,10 @@ public class ExchangeOperator implements SourceOperator {
return sourceId;
}
+ public ISourceHandle getSourceHandle() {
+ return sourceHandle;
+ }
+
@Override
public ListenableFuture<?> isBlocked() {
// Avoid registering a new callback in the source handle when one is already pending
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/SeriesScanOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/SeriesScanOperator.java
index 681ffb2628..372e33ba67 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/SeriesScanOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/SeriesScanOperator.java
@@ -19,7 +19,9 @@
package org.apache.iotdb.db.mpp.execution.operator.source;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.operator.OperatorContext;
+import org.apache.iotdb.db.mpp.execution.operator.factory.SourceOperatorFactory;
import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -35,17 +37,89 @@ import java.io.IOException;
import java.util.Set;
import java.util.concurrent.TimeUnit;
+import static com.google.common.base.Preconditions.checkState;
+import static java.util.Objects.requireNonNull;
+
public class SeriesScanOperator extends AbstractDataSourceOperator {
+ public static class SeriesScanOperatorFactory implements SourceOperatorFactory {
+ private final int operatorId;
+ private final PlanNodeId sourceId;
+ private final PartialPath seriesPath;
+ private final Set<String> allSensors;
+ private final TSDataType dataType;
+ private final Filter timeFilter;
+ private final Filter valueFilter;
+ private final boolean ascending;
+ private boolean closed;
+
+ public SeriesScanOperatorFactory(
+ int operatorId,
+ PlanNodeId sourceId,
+ PartialPath seriesPath,
+ Set<String> allSensors,
+ TSDataType dataType,
+ Filter timeFilter,
+ Filter valueFilter,
+ boolean ascending) {
+ this.operatorId = operatorId;
+ this.sourceId = requireNonNull(sourceId, "sourceId is null");
+ this.seriesPath = requireNonNull(seriesPath, "seriesPath is null");
+ this.allSensors = requireNonNull(allSensors, "allSensors is null");
+ this.dataType = requireNonNull(dataType, "dataType is null");
+ this.timeFilter = timeFilter;
+ this.valueFilter = valueFilter;
+ this.ascending = ascending;
+ }
+
+ public int getOperatorId() {
+ return operatorId;
+ }
+
+ @Override
+ public PlanNodeId getSourceId() {
+ return sourceId;
+ }
+
+ public PlanNodeId getPlanNodeId() {
+ return sourceId;
+ }
+
+ public String getOperatorType() {
+ return SeriesScanOperator.class.getSimpleName();
+ }
+
+ @Override
+ public SourceOperator createOperator(DriverContext driverContext) {
+ checkState(!closed, "Factory is already closed");
+ OperatorContext operatorContext =
+ driverContext.addOperatorContext(operatorId, sourceId, getOperatorType());
+ return new SeriesScanOperator(
+ operatorContext,
+ sourceId,
+ seriesPath,
+ allSensors,
+ dataType,
+ timeFilter,
+ valueFilter,
+ ascending);
+ }
+
+ @Override
+ public void noMoreOperators() {
+ closed = true;
+ }
+ }
+
private final TsBlockBuilder builder;
private boolean finished = false;
public SeriesScanOperator(
+ OperatorContext context,
PlanNodeId sourceId,
PartialPath seriesPath,
Set<String> allSensors,
TSDataType dataType,
- OperatorContext context,
Filter timeFilter,
Filter valueFilter,
boolean ascending) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/SeriesScanUtil.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/SeriesScanUtil.java
index e38bf71f4a..2e7505dd9b 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/SeriesScanUtil.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/SeriesScanUtil.java
@@ -23,7 +23,6 @@ import org.apache.iotdb.commons.utils.TestOnly;
import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.metadata.idtable.IDTable;
-import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.metric.QueryMetricsManager;
import org.apache.iotdb.db.query.context.QueryContext;
import org.apache.iotdb.db.query.reader.chunk.MemAlignedPageReader;
@@ -67,7 +66,8 @@ import static org.apache.iotdb.db.mpp.metric.SeriesScanCostMetricSet.BUILD_TSBLO
import static org.apache.iotdb.db.mpp.metric.SeriesScanCostMetricSet.BUILD_TSBLOCK_FROM_PAGE_READER_NONALIGNED_MEM;
public class SeriesScanUtil {
- private final FragmentInstanceContext context;
+
+ private final QueryContext context;
// The path of the target series which will be scanned.
private final PartialPath seriesPath;
@@ -135,7 +135,7 @@ public class SeriesScanUtil {
PartialPath seriesPath,
Set<String> allSensors,
TSDataType dataType,
- FragmentInstanceContext context,
+ QueryContext context,
Filter timeFilter,
Filter valueFilter,
boolean ascending) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/AbstractDriverThread.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/AbstractDriverThread.java
index 40bec7d3d6..72f5ce64a8 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/AbstractDriverThread.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/AbstractDriverThread.java
@@ -69,16 +69,15 @@ public abstract class AbstractDriverThread extends Thread implements Closeable {
continue;
}
- try (SetThreadName fragmentInstanceName =
- new SetThreadName(next.getFragmentInstance().getInfo().getFullId())) {
+ try (SetThreadName driverTaskName = new SetThreadName(next.getDriverTaskId().getFullId())) {
execute(next);
} catch (Throwable t) {
// try-with-resource syntax will call close once after try block is done, so we need to
// reset the thread name here
- try (SetThreadName fragmentInstanceName =
- new SetThreadName(next.getFragmentInstance().getInfo().getFullId())) {
+ try (SetThreadName driverTaskName =
+ new SetThreadName(next.getDriver().getDriverTaskId().getFullId())) {
logger.warn("[ExecuteFailed]", t);
- next.setAbortCause(FragmentInstanceAbortedException.BY_INTERNAL_ERROR_SCHEDULED);
+ next.setAbortCause(DriverTaskAbortedException.BY_INTERNAL_ERROR_SCHEDULED);
scheduler.toAborted(next);
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverScheduler.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverScheduler.java
index e40966ef67..4577f16809 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverScheduler.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverScheduler.java
@@ -32,7 +32,7 @@ import org.apache.iotdb.db.mpp.execution.schedule.queue.IndexedBlockingQueue;
import org.apache.iotdb.db.mpp.execution.schedule.queue.L1PriorityQueue;
import org.apache.iotdb.db.mpp.execution.schedule.queue.L2PriorityQueue;
import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTask;
-import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTaskID;
+import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTaskId;
import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTaskStatus;
import org.apache.iotdb.db.mpp.metric.QueryMetricsManager;
import org.apache.iotdb.db.utils.SetThreadName;
@@ -67,7 +67,7 @@ public class DriverScheduler implements IDriverScheduler, IService {
private final IndexedBlockingQueue<DriverTask> readyQueue;
private final IndexedBlockingQueue<DriverTask> timeoutQueue;
private final Set<DriverTask> blockedTasks;
- private final Map<QueryId, Set<DriverTask>> queryMap;
+ private final Map<QueryId, Map<FragmentInstanceId, Set<DriverTask>>> queryMap;
private final ITaskScheduler scheduler;
private IMPPDataExchangeManager blockManager;
@@ -161,17 +161,22 @@ public class DriverScheduler implements IDriverScheduler, IService {
}
@Override
- public void submitDrivers(QueryId queryId, List<IDriver> instances, long timeOut) {
+ public void submitDrivers(QueryId queryId, List<IDriver> drivers, long timeOut) {
List<DriverTask> tasks =
- instances.stream()
+ drivers.stream()
.map(
v ->
new DriverTask(
v, timeOut > 0 ? timeOut : QUERY_TIMEOUT_MS, DriverTaskStatus.READY))
.collect(Collectors.toList());
- queryMap
- .computeIfAbsent(queryId, v -> Collections.synchronizedSet(new HashSet<>()))
- .addAll(tasks);
+ for (DriverTask driverTask : tasks) {
+ queryMap
+ .computeIfAbsent(queryId, v -> new ConcurrentHashMap<>())
+ .computeIfAbsent(
+ driverTask.getDriverTaskId().getFragmentInstanceId(),
+ v -> Collections.synchronizedSet(new HashSet<>()))
+ .add(driverTask);
+ }
for (DriverTask task : tasks) {
task.lock();
try {
@@ -189,15 +194,19 @@ public class DriverScheduler implements IDriverScheduler, IService {
@Override
public void abortQuery(QueryId queryId) {
- Set<DriverTask> queryRelatedTasks = queryMap.remove(queryId);
+ Map<FragmentInstanceId, Set<DriverTask>> queryRelatedTasks = queryMap.remove(queryId);
if (queryRelatedTasks != null) {
- for (DriverTask task : queryRelatedTasks) {
- task.lock();
- try {
- task.setAbortCause(FragmentInstanceAbortedException.BY_QUERY_CASCADING_ABORTED);
- clearDriverTask(task);
- } finally {
- task.unlock();
+ for (Set<DriverTask> fragmentRelatedTasks : queryRelatedTasks.values()) {
+ if (fragmentRelatedTasks != null) {
+ for (DriverTask task : fragmentRelatedTasks) {
+ task.lock();
+ try {
+ task.setAbortCause(DriverTaskAbortedException.BY_QUERY_CASCADING_ABORTED);
+ clearDriverTask(task);
+ } finally {
+ task.unlock();
+ }
+ }
}
}
}
@@ -205,51 +214,63 @@ public class DriverScheduler implements IDriverScheduler, IService {
@Override
public void abortFragmentInstance(FragmentInstanceId instanceId) {
- DriverTask task = timeoutQueue.get(new DriverTaskID(instanceId));
- if (task == null) {
- return;
- }
- task.lock();
- try {
- task.setAbortCause(FragmentInstanceAbortedException.BY_FRAGMENT_ABORT_CALLED);
- clearDriverTask(task);
- } finally {
- task.unlock();
+ Set<DriverTask> instanceRelatedTasks = queryMap.get(instanceId.getQueryId()).remove(instanceId);
+ if (instanceRelatedTasks != null) {
+ for (DriverTask task : instanceRelatedTasks) {
+ if (task == null) {
+ return;
+ }
+ task.lock();
+ try {
+ task.setAbortCause(DriverTaskAbortedException.BY_FRAGMENT_ABORT_CALLED);
+ clearDriverTask(task);
+ } finally {
+ task.unlock();
+ }
+ }
}
}
@Override
- public double getSchedulePriority(FragmentInstanceId instanceId) {
- DriverTask task = timeoutQueue.get(new DriverTaskID(instanceId));
+ public double getSchedulePriority(DriverTaskId driverTaskID) {
+ DriverTask task = timeoutQueue.get(driverTaskID);
if (task == null) {
throw new IllegalStateException(
- "the fragmentInstance " + instanceId.getFullId() + " has been cleared");
+ "the fragmentInstance " + driverTaskID.getFullId() + " has been cleared");
}
return task.getSchedulePriority();
}
private void clearDriverTask(DriverTask task) {
- try (SetThreadName fragmentInstanceName =
- new SetThreadName(task.getFragmentInstance().getInfo().getFullId())) {
+ try (SetThreadName driverTaskName =
+ new SetThreadName(task.getDriver().getDriverTaskId().getFullId())) {
if (task.getStatus() != DriverTaskStatus.FINISHED) {
task.setStatus(DriverTaskStatus.ABORTED);
}
- readyQueue.remove(task.getId());
- timeoutQueue.remove(task.getId());
+ readyQueue.remove(task.getDriverTaskId());
+ timeoutQueue.remove(task.getDriverTaskId());
blockedTasks.remove(task);
- Set<DriverTask> tasks = queryMap.get(task.getId().getQueryId());
- if (tasks != null) {
- tasks.remove(task);
- if (tasks.isEmpty()) {
- queryMap.remove(task.getId().getQueryId());
+ Map<FragmentInstanceId, Set<DriverTask>> queryRelatedTasks =
+ queryMap.get(task.getDriverTaskId().getQueryId());
+ if (queryRelatedTasks != null) {
+ Set<DriverTask> instanceRelatedTasks =
+ queryRelatedTasks.get(task.getDriverTaskId().getFragmentInstanceId());
+ if (instanceRelatedTasks != null) {
+ instanceRelatedTasks.remove(task);
+ if (instanceRelatedTasks.isEmpty()) {
+ queryRelatedTasks.remove(task.getDriverTaskId().getFragmentInstanceId());
+ }
+ }
+ if (queryRelatedTasks.isEmpty()) {
+ queryMap.remove(task.getDriverTaskId().getQueryId());
}
}
if (task.getAbortCause() != null) {
try {
- task.getFragmentInstance()
+ task.getDriver()
.failed(
- new FragmentInstanceAbortedException(
- task.getFragmentInstance().getInfo(), task.getAbortCause()));
+ new DriverTaskAbortedException(
+ task.getDriver().getDriverTaskId().getFullId(), task.getAbortCause()));
} catch (Exception e) {
logger.error("Clear DriverTask failed", e);
}
@@ -258,9 +279,9 @@ public class DriverScheduler implements IDriverScheduler, IService {
try {
blockManager.forceDeregisterFragmentInstance(
new TFragmentInstanceId(
- task.getId().getQueryId().getId(),
- task.getId().getFragmentId().getId(),
- task.getId().getInstanceId()));
+ task.getDriverTaskId().getQueryId().getId(),
+ task.getDriverTaskId().getFragmentId().getId(),
+ task.getDriverTaskId().getFragmentInstanceId().getInstanceId()));
} catch (Exception e) {
logger.error("Clear DriverTask failed", e);
}
@@ -296,7 +317,7 @@ public class DriverScheduler implements IDriverScheduler, IService {
}
@TestOnly
- Map<QueryId, Set<DriverTask>> getQueryMap() {
+ Map<QueryId, Map<FragmentInstanceId, Set<DriverTask>>> getQueryMap() {
return queryMap;
}
@@ -398,8 +419,8 @@ public class DriverScheduler implements IDriverScheduler, IService {
@Override
public void toAborted(DriverTask task) {
- try (SetThreadName fragmentInstanceName =
- new SetThreadName(task.getFragmentInstance().getInfo().getFullId())) {
+ try (SetThreadName driverTaskName =
+ new SetThreadName(task.getDriver().getDriverTaskId().getFullId())) {
task.lock();
try {
// If a task is already in an end state, it indicates that the task is finalized in other
@@ -409,24 +430,28 @@ public class DriverScheduler implements IDriverScheduler, IService {
}
logger.warn(
"The task {} is aborted. All other tasks in the same query will be cancelled",
- task.getId());
+ task.getDriverTaskId());
clearDriverTask(task);
} finally {
task.unlock();
}
- QueryId queryId = task.getId().getQueryId();
- Set<DriverTask> queryRelatedTasks = queryMap.remove(queryId);
+ QueryId queryId = task.getDriverTaskId().getQueryId();
+ Map<FragmentInstanceId, Set<DriverTask>> queryRelatedTasks = queryMap.get(queryId);
if (queryRelatedTasks != null) {
- for (DriverTask otherTask : queryRelatedTasks) {
- if (task.equals(otherTask)) {
- continue;
- }
- otherTask.lock();
- try {
- otherTask.setAbortCause(FragmentInstanceAbortedException.BY_QUERY_CASCADING_ABORTED);
- clearDriverTask(otherTask);
- } finally {
- otherTask.unlock();
+ for (Set<DriverTask> fragmentRelatedTasks : queryRelatedTasks.values()) {
+ if (fragmentRelatedTasks != null) {
+ for (DriverTask otherTask : fragmentRelatedTasks) {
+ if (task.equals(otherTask)) {
+ continue;
+ }
+ otherTask.lock();
+ try {
+ otherTask.setAbortCause(DriverTaskAbortedException.BY_QUERY_CASCADING_ABORTED);
+ clearDriverTask(otherTask);
+ } finally {
+ otherTask.unlock();
+ }
+ }
}
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/FragmentInstanceAbortedException.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskAbortedException.java
similarity index 82%
rename from server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/FragmentInstanceAbortedException.java
rename to server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskAbortedException.java
index 6bbc447c3c..cfbb2be821 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/FragmentInstanceAbortedException.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskAbortedException.java
@@ -16,13 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.mpp.execution.schedule;
-import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.execution.driver.IDriver;
/** A common exception to pass to {@link IDriver#failed(Throwable)} */
-public class FragmentInstanceAbortedException extends Exception {
+public class DriverTaskAbortedException extends Exception {
public static final String BY_TIMEOUT = "timeout";
public static final String BY_FRAGMENT_ABORT_CALLED = " called";
@@ -30,7 +30,7 @@ public class FragmentInstanceAbortedException extends Exception {
public static final String BY_ALREADY_BEING_CANCELLED = "already being cancelled";
public static final String BY_INTERNAL_ERROR_SCHEDULED = "internal error scheduled";
- public FragmentInstanceAbortedException(FragmentInstanceId id, String causeMsg) {
- super(String.format("FragmentInstance %s is aborted by %s", id.toString(), causeMsg));
+ public DriverTaskAbortedException(String driverTaskName, String causeMsg) {
+ super(String.format("DriverTask %s is aborted by %s", driverTaskName, causeMsg));
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskThread.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskThread.java
index 28324a0502..3dd977a9ee 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskThread.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskThread.java
@@ -59,21 +59,21 @@ public class DriverTaskThread extends AbstractDriverThread {
if (!scheduler.readyToRunning(task)) {
return;
}
- IDriver instance = task.getFragmentInstance();
+ IDriver driver = task.getDriver();
CpuTimer timer = new CpuTimer();
- ListenableFuture<?> future = instance.processFor(EXECUTION_TIME_SLICE);
+ ListenableFuture<?> future = driver.processFor(EXECUTION_TIME_SLICE);
CpuTimer.CpuDuration duration = timer.elapsedTime();
// long cost = System.nanoTime() - startTime;
// If the future is cancelled, the task is in an error and should be thrown.
if (future.isCancelled()) {
- task.setAbortCause(FragmentInstanceAbortedException.BY_ALREADY_BEING_CANCELLED);
+ task.setAbortCause(DriverTaskAbortedException.BY_ALREADY_BEING_CANCELLED);
scheduler.toAborted(task);
return;
}
ExecutionContext context = new ExecutionContext();
context.setCpuDuration(duration);
context.setTimeSlice(EXECUTION_TIME_SLICE);
- if (instance.isFinished()) {
+ if (driver.isFinished()) {
scheduler.runningToFinished(task, context);
return;
}
@@ -84,8 +84,8 @@ public class DriverTaskThread extends AbstractDriverThread {
scheduler.runningToBlocked(task, context);
future.addListener(
() -> {
- try (SetThreadName fragmentInstanceName2 =
- new SetThreadName(task.getFragmentInstance().getInfo().getFullId())) {
+ try (SetThreadName driverTaskName2 =
+ new SetThreadName(task.getDriver().getDriverTaskId().getFullId())) {
scheduler.blockedToReady(task);
}
},
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskTimeoutSentinelThread.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskTimeoutSentinelThread.java
index a9533685a1..ed41dbe96f 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskTimeoutSentinelThread.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskTimeoutSentinelThread.java
@@ -73,7 +73,7 @@ public class DriverTaskTimeoutSentinelThread extends AbstractDriverThread {
"[DriverTaskTimeout] Current time is {}, ddl of task is {}",
System.currentTimeMillis(),
task.getDDL());
- task.setAbortCause(FragmentInstanceAbortedException.BY_TIMEOUT);
+ task.setAbortCause(DriverTaskAbortedException.BY_TIMEOUT);
scheduler.toAborted(task);
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/IDriverScheduler.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/IDriverScheduler.java
index 7939c25a25..93dd3e314c 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/IDriverScheduler.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/IDriverScheduler.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.mpp.execution.schedule;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.QueryId;
import org.apache.iotdb.db.mpp.execution.driver.IDriver;
+import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTaskId;
import java.util.List;
@@ -52,11 +53,11 @@ public interface IDriverScheduler {
void abortFragmentInstance(FragmentInstanceId instanceId);
/**
- * Return the schedule priority of a fragment.
+ * Return the schedule priority of a Driver task.
*
- * @param instanceId the fragment instance id.
+ * @param driverTaskID the fragment instance id.
* @return the schedule priority.
* @throws IllegalStateException if the instance has already been cleared.
*/
- double getSchedulePriority(FragmentInstanceId instanceId);
+ double getSchedulePriority(DriverTaskId driverTaskID);
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/queue/IDIndexedAccessible.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/queue/IDIndexedAccessible.java
index 54dd5dc71a..e45555e446 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/queue/IDIndexedAccessible.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/queue/IDIndexedAccessible.java
@@ -42,7 +42,7 @@ package org.apache.iotdb.db.mpp.execution.schedule.queue;
*/
public interface IDIndexedAccessible {
- ID getId();
+ ID getDriverTaskId();
void setId(ID id);
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/queue/IndexedBlockingQueue.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/queue/IndexedBlockingQueue.java
index 219008350d..1498bbbc30 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/queue/IndexedBlockingQueue.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/queue/IndexedBlockingQueue.java
@@ -86,7 +86,8 @@ public abstract class IndexedBlockingQueue<E extends IDIndexedAccessible> {
throw new NullPointerException("pushed element is null");
}
Preconditions.checkState(
- !contains(element), "The queue has already contained the element: " + element.getId());
+ !contains(element),
+ "The queue has already contained the element: " + element.getDriverTaskId());
Preconditions.checkState(size < MAX_CAPACITY, "The queue is full");
pushToQueue(element);
size++;
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/queue/L1PriorityQueue.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/queue/L1PriorityQueue.java
index ab8aff713b..fc1f88a854 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/queue/L1PriorityQueue.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/queue/L1PriorityQueue.java
@@ -65,19 +65,19 @@ public class L1PriorityQueue<E extends IDIndexedAccessible> extends IndexedBlock
protected E pollFirst() {
E element = sortedElements.first();
sortedElements.remove(element);
- keyedElements.remove(element.getId());
+ keyedElements.remove(element.getDriverTaskId());
return element;
}
@Override
protected void pushToQueue(E element) {
- keyedElements.put(element.getId(), element);
+ keyedElements.put(element.getDriverTaskId(), element);
sortedElements.add(element);
}
@Override
protected E remove(E element) {
- E e = keyedElements.remove(element.getId());
+ E e = keyedElements.remove(element.getDriverTaskId());
if (e != null) {
sortedElements.remove(e);
}
@@ -86,12 +86,12 @@ public class L1PriorityQueue<E extends IDIndexedAccessible> extends IndexedBlock
@Override
protected boolean contains(E element) {
- return keyedElements.containsKey(element.getId());
+ return keyedElements.containsKey(element.getDriverTaskId());
}
@Override
protected E get(E element) {
- return keyedElements.get(element.getId());
+ return keyedElements.get(element.getDriverTaskId());
}
@Override
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/queue/L2PriorityQueue.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/queue/L2PriorityQueue.java
index 6f2a802b4b..1dc4b27003 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/queue/L2PriorityQueue.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/queue/L2PriorityQueue.java
@@ -80,24 +80,24 @@ public class L2PriorityQueue<E extends IDIndexedAccessible> extends IndexedBlock
}
E element = workingSortedElements.first();
workingSortedElements.remove(element);
- workingKeyedElements.remove(element.getId());
+ workingKeyedElements.remove(element.getDriverTaskId());
return element;
}
@Override
protected void pushToQueue(E element) {
- idleKeyedElements.put(element.getId(), element);
+ idleKeyedElements.put(element.getDriverTaskId(), element);
idleSortedElements.add(element);
}
@Override
protected E remove(E element) {
- E e = workingKeyedElements.remove(element.getId());
+ E e = workingKeyedElements.remove(element.getDriverTaskId());
if (e != null) {
workingSortedElements.remove(e);
return e;
}
- e = idleKeyedElements.remove(element.getId());
+ e = idleKeyedElements.remove(element.getDriverTaskId());
if (e != null) {
idleSortedElements.remove(e);
}
@@ -106,17 +106,17 @@ public class L2PriorityQueue<E extends IDIndexedAccessible> extends IndexedBlock
@Override
protected boolean contains(E element) {
- return workingKeyedElements.containsKey(element.getId())
- || idleKeyedElements.containsKey(element.getId());
+ return workingKeyedElements.containsKey(element.getDriverTaskId())
+ || idleKeyedElements.containsKey(element.getDriverTaskId());
}
@Override
protected E get(E element) {
- E e = workingKeyedElements.get(element.getId());
+ E e = workingKeyedElements.get(element.getDriverTaskId());
if (e != null) {
return e;
}
- return idleKeyedElements.get(element.getId());
+ return idleKeyedElements.get(element.getDriverTaskId());
}
@Override
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/task/DriverTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/task/DriverTask.java
index 469159beaf..ef173a45be 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/task/DriverTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/task/DriverTask.java
@@ -39,9 +39,8 @@ import java.util.concurrent.locks.ReentrantLock;
/** the scheduling element of {@link DriverTaskThread}. It wraps a single Driver. */
public class DriverTask implements IDIndexedAccessible {
- private DriverTaskID id;
+ private final IDriver driver;
private DriverTaskStatus status;
- private final IDriver fragmentInstance;
// the higher this field is, the higher probability it will be scheduled.
private volatile double schedulePriority;
@@ -61,22 +60,21 @@ public class DriverTask implements IDIndexedAccessible {
this(new StubFragmentInstance(), 0L, null);
}
- public DriverTask(IDriver instance, long timeoutMs, DriverTaskStatus status) {
- this.fragmentInstance = instance;
- this.id = new DriverTaskID(instance.getInfo());
+ public DriverTask(IDriver driver, long timeoutMs, DriverTaskStatus status) {
+ this.driver = driver;
this.setStatus(status);
this.schedulePriority = 0.0D;
this.ddl = System.currentTimeMillis() + timeoutMs;
this.lock = new ReentrantLock();
}
- public DriverTaskID getId() {
- return id;
+ public DriverTaskId getDriverTaskId() {
+ return driver.getDriverTaskId();
}
@Override
public void setId(ID id) {
- this.id = (DriverTaskID) id;
+ driver.setDriverTaskId((DriverTaskId) id);
}
public DriverTaskStatus getStatus() {
@@ -87,8 +85,8 @@ public class DriverTask implements IDIndexedAccessible {
return status == DriverTaskStatus.ABORTED || status == DriverTaskStatus.FINISHED;
}
- public IDriver getFragmentInstance() {
- return fragmentInstance;
+ public IDriver getDriver() {
+ return driver;
}
public void setStatus(DriverTaskStatus status) {
@@ -133,12 +131,12 @@ public class DriverTask implements IDIndexedAccessible {
@Override
public int hashCode() {
- return id.hashCode();
+ return driver.getDriverTaskId().hashCode();
}
@Override
public boolean equals(Object o) {
- return o instanceof DriverTask && ((DriverTask) o).getId().equals(id);
+ return o instanceof DriverTask && ((DriverTask) o).getDriverTaskId().equals(getDriverTaskId());
}
public String getAbortCause() {
@@ -170,7 +168,7 @@ public class DriverTask implements IDIndexedAccessible {
@Override
public int compare(DriverTask o1, DriverTask o2) {
- if (o1.getId().equals(o2.getId())) {
+ if (o1.getDriverTaskId().equals(o2.getDriverTaskId())) {
return 0;
}
if (o1.getDDL() < o2.getDDL()) {
@@ -179,7 +177,7 @@ public class DriverTask implements IDIndexedAccessible {
if (o1.getDDL() > o2.getDDL()) {
return 1;
}
- return o1.getId().compareTo(o2.getId());
+ return o1.getDriverTaskId().compareTo(o2.getDriverTaskId());
}
}
@@ -188,7 +186,7 @@ public class DriverTask implements IDIndexedAccessible {
@Override
public int compare(DriverTask o1, DriverTask o2) {
- if (o1.getId().equals(o2.getId())) {
+ if (o1.getDriverTaskId().equals(o2.getDriverTaskId())) {
return 0;
}
if (o1.getSchedulePriority() > o2.getSchedulePriority()) {
@@ -197,15 +195,16 @@ public class DriverTask implements IDIndexedAccessible {
if (o1.getSchedulePriority() < o2.getSchedulePriority()) {
return 1;
}
- return o1.getId().compareTo(o2.getId());
+ return o1.getDriverTaskId().compareTo(o2.getDriverTaskId());
}
}
private static class StubFragmentInstance implements IDriver {
private static final QueryId stubQueryId = new QueryId("stub_query");
- private static final FragmentInstanceId stubInstance =
- new FragmentInstanceId(new PlanFragmentId(stubQueryId, 0), "stub-instance");
+ private static DriverTaskId stubDriver =
+ new DriverTaskId(
+ new FragmentInstanceId(new PlanFragmentId(stubQueryId, 0), "stub-instance"), 0);
@Override
public boolean isFinished() {
@@ -218,8 +217,13 @@ public class DriverTask implements IDIndexedAccessible {
}
@Override
- public FragmentInstanceId getInfo() {
- return stubInstance;
+ public DriverTaskId getDriverTaskId() {
+ return stubDriver;
+ }
+
+ @Override
+ public void setDriverTaskId(DriverTaskId driverTaskId) {
+ stubDriver = driverTaskId;
}
@Override
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/task/DriverTaskID.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/task/DriverTaskId.java
similarity index 56%
rename from server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/task/DriverTaskID.java
rename to server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/task/DriverTaskId.java
index 9f7b7227ff..e37fd8cbb2 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/task/DriverTaskID.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/task/DriverTaskId.java
@@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.mpp.execution.schedule.task;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
@@ -25,44 +26,62 @@ import org.apache.iotdb.db.mpp.execution.schedule.queue.ID;
import org.jetbrains.annotations.NotNull;
-/** the class of id of the fragment instance task */
-public class DriverTaskID implements ID, Comparable<DriverTaskID> {
+import java.util.Objects;
+
+/** the class of id of the pipeline driver task */
+public class DriverTaskId implements ID, Comparable<DriverTaskId> {
- private final FragmentInstanceId id;
+ private final FragmentInstanceId fragmentInstanceId;
+ // TODO Create another field to store id of driver level
+ // Currently, we just save pipelineId in driverTask since it's one-to-one relation.
+ private final int pipelineId;
+ private final String fullId;
- public DriverTaskID(FragmentInstanceId id) {
- this.id = id;
+ public DriverTaskId(FragmentInstanceId id, int pipelineId) {
+ this.fragmentInstanceId = id;
+ this.pipelineId = pipelineId;
+ this.fullId = String.format("%s.%d", id.getFullId(), pipelineId);
}
@Override
- public boolean equals(Object o) {
- return o instanceof DriverTaskID && ((DriverTaskID) o).id.equals(id);
+ public int hashCode() {
+ return Objects.hash(fragmentInstanceId, pipelineId);
}
@Override
- public int hashCode() {
- return id.hashCode();
+ public boolean equals(Object o) {
+ return o instanceof DriverTaskId
+ && ((DriverTaskId) o).fragmentInstanceId.equals(fragmentInstanceId)
+ && ((DriverTaskId) o).pipelineId == pipelineId;
}
public String toString() {
- return id.getFullId();
+ return fullId;
}
- public String getInstanceId() {
- return id.getInstanceId();
+ public FragmentInstanceId getFragmentInstanceId() {
+ return fragmentInstanceId;
}
public PlanFragmentId getFragmentId() {
- return id.getFragmentId();
+ return fragmentInstanceId.getFragmentId();
}
public QueryId getQueryId() {
- return id.getQueryId();
+ return fragmentInstanceId.getQueryId();
+ }
+
+ public int getPipelineId() {
+ return pipelineId;
+ }
+
+ public String getFullId() {
+ return fullId;
}
// This is the default comparator of FragmentInstanceID
@Override
- public int compareTo(@NotNull DriverTaskID o) {
+ public int compareTo(@NotNull DriverTaskId o) {
return String.CASE_INSENSITIVE_ORDER.compare(this.toString(), o.toString());
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/QueryExecution.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/QueryExecution.java
index e8e2afa40c..9cf25c433c 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/QueryExecution.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/QueryExecution.java
@@ -542,7 +542,7 @@ public class QueryExecution implements IQueryExecution {
isSameNode(upstreamEndPoint)
? MPPDataExchangeService.getInstance()
.getMPPDataExchangeManager()
- .createLocalSourceHandle(
+ .createLocalSourceHandleForFragment(
context.getResultNodeContext().getVirtualFragmentInstanceId().toThrift(),
context.getResultNodeContext().getVirtualResultNodeId().getId(),
context.getResultNodeContext().getUpStreamFragmentInstanceId().toThrift(),
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LocalExecutionPlanContext.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LocalExecutionPlanContext.java
index 351303134f..2d8e6d7051 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LocalExecutionPlanContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LocalExecutionPlanContext.java
@@ -18,10 +18,16 @@
*/
package org.apache.iotdb.db.mpp.plan.planner;
-import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.metadata.schemaregion.ISchemaRegion;
+import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
+import org.apache.iotdb.db.mpp.execution.driver.DataDriverContext;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
+import org.apache.iotdb.db.mpp.execution.driver.SchemaDriverContext;
import org.apache.iotdb.db.mpp.execution.exchange.ISinkHandle;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
-import org.apache.iotdb.db.mpp.execution.operator.source.DataSourceOperator;
+import org.apache.iotdb.db.mpp.execution.operator.Operator;
+import org.apache.iotdb.db.mpp.execution.operator.source.ExchangeOperator;
import org.apache.iotdb.db.mpp.execution.timer.RuleBasedTimeSliceAllocator;
import org.apache.iotdb.db.mpp.plan.analyze.TypeProvider;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -30,31 +36,36 @@ import org.apache.iotdb.tsfile.read.filter.basic.Filter;
import org.apache.iotdb.tsfile.utils.Binary;
import org.apache.iotdb.tsfile.utils.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
import java.util.ArrayList;
-import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
-import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
import static com.google.common.base.Preconditions.checkArgument;
import static java.util.Objects.requireNonNull;
+// Attention: We should use thread-safe data structure for members that are shared by all pipelines
public class LocalExecutionPlanContext {
- private final FragmentInstanceContext instanceContext;
- private final List<PartialPath> paths;
- // deviceId -> sensorId Set
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(LocalExecutionPlanContext.class);
+ // Save operators in this pipeline, a new one will be created when creating another pipeline
+ private final DriverContext driverContext;
+ private final AtomicInteger nextOperatorId;
+ private final TypeProvider typeProvider;
private final Map<String, Set<String>> allSensorsMap;
- // Used to lock corresponding query resources
- private final List<DataSourceOperator> sourceOperators;
+ // this is shared with all subContexts
+ private AtomicInteger nextPipelineId;
+ private List<PipelineDriverFactory> pipelineDriverFactories;
+ private List<ExchangeOperator> exchangeOperatorList = new ArrayList<>();
+ private int exchangeSumNum = 0;
private final long dataRegionTTL;
- private ISinkHandle sinkHandle;
-
- private int nextOperatorId = 0;
-
- private final TypeProvider typeProvider;
private List<TSDataType> cachedDataTypes;
@@ -66,58 +77,119 @@ public class LocalExecutionPlanContext {
// whether we need to update last cache
private boolean needUpdateLastCache;
- private final RuleBasedTimeSliceAllocator timeSliceAllocator;
-
// for data region
public LocalExecutionPlanContext(
- TypeProvider typeProvider, FragmentInstanceContext instanceContext, long dataRegionTTL) {
+ TypeProvider typeProvider, FragmentInstanceContext instanceContext) {
this.typeProvider = typeProvider;
- this.instanceContext = instanceContext;
- this.paths = new ArrayList<>();
- this.allSensorsMap = new HashMap<>();
- this.sourceOperators = new ArrayList<>();
- this.timeSliceAllocator = new RuleBasedTimeSliceAllocator();
- this.dataRegionTTL = dataRegionTTL;
+ this.allSensorsMap = new ConcurrentHashMap<>();
+ this.dataRegionTTL = instanceContext.getDataRegion().getDataTTL();
+ this.nextOperatorId = new AtomicInteger(0);
+ this.nextPipelineId = new AtomicInteger(0);
+ this.driverContext = new DataDriverContext(instanceContext, getNextPipelineId());
+ this.pipelineDriverFactories = new ArrayList<>();
+ }
+
+ // For creating subContext, differ from parent context mainly in driver context
+ public LocalExecutionPlanContext(LocalExecutionPlanContext parentContext) {
+ this.nextOperatorId = parentContext.nextOperatorId;
+ this.typeProvider = parentContext.typeProvider;
+ this.allSensorsMap = parentContext.allSensorsMap;
+ this.dataRegionTTL = parentContext.dataRegionTTL;
+ this.nextPipelineId = parentContext.nextPipelineId;
+ this.pipelineDriverFactories = parentContext.pipelineDriverFactories;
+ this.exchangeSumNum = parentContext.exchangeSumNum;
+ this.exchangeOperatorList = parentContext.exchangeOperatorList;
+ this.cachedDataTypes = parentContext.cachedDataTypes;
+ this.driverContext =
+ parentContext.getDriverContext().createSubDriverContext(getNextPipelineId());
}
// for schema region
- public LocalExecutionPlanContext(FragmentInstanceContext instanceContext) {
- this.instanceContext = instanceContext;
- this.paths = new ArrayList<>();
- this.allSensorsMap = new HashMap<>();
- this.sourceOperators = new ArrayList<>();
+ public LocalExecutionPlanContext(
+ FragmentInstanceContext instanceContext, ISchemaRegion schemaRegion) {
+ this.allSensorsMap = new ConcurrentHashMap<>();
this.typeProvider = null;
+ this.nextOperatorId = new AtomicInteger(0);
- // only used in `order by heat`
- this.timeSliceAllocator = new RuleBasedTimeSliceAllocator();
// there is no ttl in schema region, so we don't care this field
this.dataRegionTTL = Long.MAX_VALUE;
+ this.driverContext = new SchemaDriverContext(instanceContext, schemaRegion);
+ }
+
+ public void addPipelineDriverFactory(Operator operation, DriverContext driverContext) {
+ driverContext
+ .getOperatorContexts()
+ .forEach(
+ operatorContext ->
+ operatorContext.setMaxRunTime(
+ driverContext.getTimeSliceAllocator().getMaxRunTime(operatorContext)));
+ pipelineDriverFactories.add(new PipelineDriverFactory(operation, driverContext));
+ }
+
+ public LocalExecutionPlanContext createSubContext() {
+ return new LocalExecutionPlanContext(this);
+ }
+
+ public FragmentInstanceId getFragmentInstanceId() {
+ return driverContext.getFragmentInstanceContext().getId();
+ }
+
+ public List<PipelineDriverFactory> getPipelineDriverFactories() {
+ return pipelineDriverFactories;
+ }
+
+ public DriverContext getDriverContext() {
+ return driverContext;
+ }
+
+ private int getNextPipelineId() {
+ return nextPipelineId.getAndIncrement();
+ }
+
+ public boolean isInputDriver() {
+ return driverContext.isInputDriver();
}
public int getNextOperatorId() {
- return nextOperatorId++;
+ return nextOperatorId.getAndIncrement();
}
- public List<PartialPath> getPaths() {
- return paths;
+ public int getExchangeSumNum() {
+ return exchangeSumNum;
}
- public Set<String> getAllSensors(String deviceId, String sensorId) {
- Set<String> allSensors = allSensorsMap.computeIfAbsent(deviceId, k -> new HashSet<>());
- allSensors.add(sensorId);
- return allSensors;
+ public void setExchangeSumNum(int exchangeSumNum) {
+ this.exchangeSumNum = exchangeSumNum;
+ }
+
+ public long getMaxBytesOneHandleCanReserve() {
+ long maxBytesPerFI = IoTDBDescriptor.getInstance().getConfig().getMaxBytesPerFragmentInstance();
+ return exchangeSumNum == 0 ? maxBytesPerFI : maxBytesPerFI / exchangeSumNum;
}
- public List<DataSourceOperator> getSourceOperators() {
- return sourceOperators;
+ public void addExchangeSumNum(int addValue) {
+ this.exchangeSumNum += addValue;
}
- public void addPath(PartialPath path) {
- paths.add(path);
+ public void addExchangeOperator(ExchangeOperator exchangeOperator) {
+ this.exchangeOperatorList.add(exchangeOperator);
}
- public void addSourceOperator(DataSourceOperator sourceOperator) {
- sourceOperators.add(sourceOperator);
+ public void setMaxBytesOneHandleCanReserve() {
+ long maxBytesOneHandleCanReserve = getMaxBytesOneHandleCanReserve();
+ LOGGER.debug(
+ "MaxBytesOneHandleCanReserve for ExchangeOperator is {}, exchangeSumNum is {}.",
+ maxBytesOneHandleCanReserve,
+ exchangeSumNum);
+ exchangeOperatorList.forEach(
+ exchangeOperator ->
+ exchangeOperator.getSourceHandle().setMaxBytesCanReserve(maxBytesOneHandleCanReserve));
+ }
+
+ public Set<String> getAllSensors(String deviceId, String sensorId) {
+ Set<String> allSensors = allSensorsMap.computeIfAbsent(deviceId, k -> new HashSet<>());
+ allSensors.add(sensorId);
+ return allSensors;
}
public void setLastQueryTimeFilter(Filter lastQueryTimeFilter) {
@@ -139,15 +211,10 @@ public class LocalExecutionPlanContext {
return cachedLastValueAndPathList;
}
- public ISinkHandle getSinkHandle() {
- return sinkHandle;
- }
-
public void setSinkHandle(ISinkHandle sinkHandle) {
requireNonNull(sinkHandle, "sinkHandle is null");
- checkArgument(this.sinkHandle == null, "There must be at most one SinkNode");
-
- this.sinkHandle = sinkHandle;
+ checkArgument(driverContext.getSinkHandle() == null, "There must be at most one SinkNode");
+ driverContext.setSinkHandle(sinkHandle);
}
public void setCachedDataTypes(List<TSDataType> cachedDataTypes) {
@@ -163,11 +230,11 @@ public class LocalExecutionPlanContext {
}
public RuleBasedTimeSliceAllocator getTimeSliceAllocator() {
- return timeSliceAllocator;
+ return driverContext.getTimeSliceAllocator();
}
public FragmentInstanceContext getInstanceContext() {
- return instanceContext;
+ return driverContext.getFragmentInstanceContext();
}
public Filter getLastQueryTimeFilter() {
@@ -181,8 +248,4 @@ public class LocalExecutionPlanContext {
public long getDataRegionTTL() {
return dataRegionTTL;
}
-
- public ExecutorService getIntoOperationExecutor() {
- return instanceContext.getIntoOperationExecutor();
- }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LocalExecutionPlanner.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LocalExecutionPlanner.java
index 66c4e717e4..231ca1d680 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LocalExecutionPlanner.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LocalExecutionPlanner.java
@@ -18,16 +18,14 @@
*/
package org.apache.iotdb.db.mpp.plan.planner;
+import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.storagegroup.IDataRegionForQuery;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.metadata.schemaregion.ISchemaRegion;
import org.apache.iotdb.db.mpp.exception.MemoryNotEnoughException;
-import org.apache.iotdb.db.mpp.execution.driver.DataDriver;
import org.apache.iotdb.db.mpp.execution.driver.DataDriverContext;
import org.apache.iotdb.db.mpp.execution.driver.SchemaDriver;
import org.apache.iotdb.db.mpp.execution.driver.SchemaDriverContext;
-import org.apache.iotdb.db.mpp.execution.exchange.ISourceHandle;
-import org.apache.iotdb.db.mpp.execution.exchange.MPPDataExchangeService;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.Operator;
@@ -35,13 +33,14 @@ import org.apache.iotdb.db.mpp.execution.timer.ITimeSliceAllocator;
import org.apache.iotdb.db.mpp.plan.analyze.TypeProvider;
import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode;
import org.apache.iotdb.db.utils.SetThreadName;
-import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstanceId;
import org.apache.iotdb.rpc.TSStatusCode;
-import org.apache.iotdb.tsfile.read.filter.basic.Filter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.util.ArrayList;
+import java.util.List;
+
/**
* Used to plan a fragment instance. Currently, we simply change it from PlanNode to executable
* Operator tree, but in the future, we may split one fragment instance into multiple pipeline to
@@ -59,89 +58,52 @@ public class LocalExecutionPlanner {
return InstanceHolder.INSTANCE;
}
- public DataDriver plan(
- PlanNode plan,
- TypeProvider types,
- FragmentInstanceContext instanceContext,
- Filter timeFilter,
- IDataRegionForQuery dataRegion)
- throws MemoryNotEnoughException {
- LocalExecutionPlanContext context =
- new LocalExecutionPlanContext(types, instanceContext, dataRegion.getDataTTL());
+ public List<PipelineDriverFactory> plan(
+ PlanNode plan, TypeProvider types, FragmentInstanceContext instanceContext)
+ throws MemoryNotEnoughException, QueryProcessException {
+ LocalExecutionPlanContext context = new LocalExecutionPlanContext(types, instanceContext);
+ // Generate pipelines, return the last pipeline data structure
+ // TODO Replace operator with operatorFactory to build multiple driver for one pipeline
Operator root = plan.accept(new OperatorTreeGenerator(), context);
// check whether current free memory is enough to execute current query
checkMemory(root, instanceContext.getStateMachine());
- // calculate memory distribution of ISinkHandle/ISourceHandle
- setMemoryLimitForHandle(instanceContext.getId().toThrift(), plan);
+ context.addPipelineDriverFactory(root, context.getDriverContext());
- ITimeSliceAllocator timeSliceAllocator = context.getTimeSliceAllocator();
- instanceContext
- .getOperatorContexts()
- .forEach(
- operatorContext ->
- operatorContext.setMaxRunTime(timeSliceAllocator.getMaxRunTime(operatorContext)));
+ List<PartialPath> sourcePaths = collectSourcePaths(context);
+ instanceContext.initQueryDataSource(sourcePaths);
+
+ // set maxBytes one SourceHandle can reserve after visiting the whole tree
+ context.setMaxBytesOneHandleCanReserve();
- DataDriverContext dataDriverContext =
- new DataDriverContext(
- instanceContext,
- context.getPaths(),
- timeFilter,
- dataRegion,
- context.getSourceOperators());
- instanceContext.setDriverContext(dataDriverContext);
- return new DataDriver(root, context.getSinkHandle(), dataDriverContext);
+ return context.getPipelineDriverFactories();
}
public SchemaDriver plan(
PlanNode plan, FragmentInstanceContext instanceContext, ISchemaRegion schemaRegion)
throws MemoryNotEnoughException {
-
- SchemaDriverContext schemaDriverContext =
- new SchemaDriverContext(instanceContext, schemaRegion);
- instanceContext.setDriverContext(schemaDriverContext);
-
- LocalExecutionPlanContext context = new LocalExecutionPlanContext(instanceContext);
+ LocalExecutionPlanContext context =
+ new LocalExecutionPlanContext(instanceContext, schemaRegion);
Operator root = plan.accept(new OperatorTreeGenerator(), context);
- // calculate memory distribution of ISinkHandle/ISourceHandle
- setMemoryLimitForHandle(instanceContext.getId().toThrift(), plan);
-
// check whether current free memory is enough to execute current query
checkMemory(root, instanceContext.getStateMachine());
+ // set maxBytes one SourceHandle can reserve after visiting the whole tree
+ context.setMaxBytesOneHandleCanReserve();
+
ITimeSliceAllocator timeSliceAllocator = context.getTimeSliceAllocator();
- instanceContext
+ context
+ .getDriverContext()
.getOperatorContexts()
.forEach(
operatorContext ->
operatorContext.setMaxRunTime(timeSliceAllocator.getMaxRunTime(operatorContext)));
- return new SchemaDriver(root, context.getSinkHandle(), schemaDriverContext);
- }
-
- private void setMemoryLimitForHandle(TFragmentInstanceId fragmentInstanceId, PlanNode plan) {
- MemoryDistributionCalculator visitor = new MemoryDistributionCalculator();
- plan.accept(visitor, null);
- int totalSplit = visitor.calculateTotalSplit();
- if (totalSplit == 0) {
- return;
- }
- long maxBytesOneHandleCanReserve =
- IoTDBDescriptor.getInstance().getConfig().getMaxBytesPerFragmentInstance() / totalSplit;
- for (ISourceHandle handle :
- MPPDataExchangeService.getInstance()
- .getMPPDataExchangeManager()
- .getISourceHandle(fragmentInstanceId)) {
- handle.setMaxBytesCanReserve(maxBytesOneHandleCanReserve);
- }
- MPPDataExchangeService.getInstance()
- .getMPPDataExchangeManager()
- .getISinkHandle(fragmentInstanceId)
- .setMaxBytesCanReserve(maxBytesOneHandleCanReserve);
+ return new SchemaDriver(root, (SchemaDriverContext) context.getDriverContext());
}
private void checkMemory(Operator root, FragmentInstanceStateMachine stateMachine)
@@ -191,6 +153,16 @@ public class LocalExecutionPlanner {
});
}
+ private List<PartialPath> collectSourcePaths(LocalExecutionPlanContext context) {
+ List<PartialPath> sourcePaths = new ArrayList<>();
+ context
+ .getPipelineDriverFactories()
+ .forEach(
+ pipeline ->
+ sourcePaths.addAll(((DataDriverContext) pipeline.getDriverContext()).getPaths()));
+ return sourcePaths;
+ }
+
private static class InstanceHolder {
private InstanceHolder() {}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/MemoryDistributionCalculator.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/MemoryDistributionCalculator.java
deleted file mode 100644
index e257554300..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/MemoryDistributionCalculator.java
+++ /dev/null
@@ -1,472 +0,0 @@
-/*
- * 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.plan.planner;
-
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanVisitor;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.CountSchemaMergeNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.DevicesCountNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.DevicesSchemaScanNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.LevelTimeSeriesCountNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.NodeManagementMemoryMergeNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.NodePathsConvertNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.NodePathsCountNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.NodePathsSchemaScanNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.SchemaFetchMergeNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.SchemaFetchScanNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.SchemaQueryMergeNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.SchemaQueryOrderByHeatNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.SchemaQueryScanNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.TimeSeriesCountNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.TimeSeriesSchemaScanNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.ConstructSchemaBlackListNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.RollbackSchemaBlackListNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.AggregationNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.DeviceMergeNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.DeviceViewIntoNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.DeviceViewNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.ExchangeNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.FillNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.FilterNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.GroupByLevelNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.GroupByTagNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.IntoNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.LimitNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.MergeSortNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.OffsetNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.ProjectNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.SingleDeviceViewNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.SlidingWindowAggregationNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.SortNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.TimeJoinNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.TransformNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.VerticallyConcatNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.last.LastQueryCollectNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.last.LastQueryMergeNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.process.last.LastQueryNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.sink.FragmentSinkNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.source.AlignedLastQueryScanNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.source.AlignedSeriesAggregationScanNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.source.AlignedSeriesScanNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.source.LastQueryScanNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.source.SeriesAggregationScanNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.source.SeriesScanNode;
-import org.apache.iotdb.db.mpp.plan.planner.plan.node.source.ShowQueriesNode;
-
-import static org.apache.iotdb.db.mpp.common.DataNodeEndPoints.isSameNode;
-
-public class MemoryDistributionCalculator
- extends PlanVisitor<Void, MemoryDistributionCalculator.MemoryDistributionContext> {
- /** This map is used to calculate the total split of memory */
- private int exchangeNum;
-
- public int calculateTotalSplit() {
- return exchangeNum;
- }
-
- @Override
- public Void visitPlan(PlanNode node, MemoryDistributionContext context) {
- // Throw exception here because we want to ensure that all new PlanNodes implement
- // this method correctly if necessary.
- throw new UnsupportedOperationException("Should call concrete visitXX method");
- }
-
- private void processConsumeChildrenOneByOneNode(PlanNode node) {
- MemoryDistributionContext context =
- new MemoryDistributionContext(
- node.getPlanNodeId(), MemoryDistributionType.CONSUME_CHILDREN_ONE_BY_ONE);
- node.getChildren()
- .forEach(
- child -> {
- if (child != null) {
- child.accept(this, context);
- }
- });
- }
-
- private void processConsumeAllChildrenAtTheSameTime(PlanNode node) {
- MemoryDistributionContext context =
- new MemoryDistributionContext(
- node.getPlanNodeId(), MemoryDistributionType.CONSUME_ALL_CHILDREN_AT_THE_SAME_TIME);
- node.getChildren()
- .forEach(
- child -> {
- if (child != null) {
- child.accept(this, context);
- }
- });
- }
-
- /** We do not distinguish LocalSourceHandle/SourceHandle by not letting LocalSinkHandle update */
- @Override
- public Void visitExchange(ExchangeNode node, MemoryDistributionContext context) {
- // context == null means this ExchangeNode doesn't have a father
- if (context == null
- || context.memoryDistributionType.equals(
- MemoryDistributionType.CONSUME_ALL_CHILDREN_AT_THE_SAME_TIME)) {
- exchangeNum++;
- } else if (!context.exchangeAdded) {
- context.exchangeAdded = true;
- exchangeNum++;
- }
- return null;
- }
-
- @Override
- public Void visitFragmentSink(FragmentSinkNode node, MemoryDistributionContext context) {
- // LocalSinkHandle and LocalSourceHandle are one-to-one mapped and only LocalSourceHandle do the
- // update
- if (!isSameNode(node.getDownStreamEndpoint())) {
- exchangeNum++;
- }
- node.getChild().accept(this, context);
- return null;
- }
-
- @Override
- public Void visitSeriesScan(SeriesScanNode node, MemoryDistributionContext context) {
- // do nothing since SourceNode will not have Exchange/FragmentSink as child
- return null;
- }
-
- @Override
- public Void visitSeriesAggregationScan(
- SeriesAggregationScanNode node, MemoryDistributionContext context) {
- // do nothing since SourceNode will not have Exchange/FragmentSink as child
- return null;
- }
-
- @Override
- public Void visitAlignedSeriesScan(
- AlignedSeriesScanNode node, MemoryDistributionContext context) {
- // do nothing since SourceNode will not have Exchange/FragmentSink as child
- return null;
- }
-
- @Override
- public Void visitAlignedSeriesAggregationScan(
- AlignedSeriesAggregationScanNode node, MemoryDistributionContext context) {
- // do nothing since SourceNode will not have Exchange/FragmentSink as child
- return null;
- }
-
- @Override
- public Void visitDeviceView(DeviceViewNode node, MemoryDistributionContext context) {
- // consume children one by one
- processConsumeChildrenOneByOneNode(node);
- return null;
- }
-
- @Override
- public Void visitDeviceMerge(DeviceMergeNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitFill(FillNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitFilter(FilterNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitGroupByLevel(GroupByLevelNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitGroupByTag(GroupByTagNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitSlidingWindowAggregation(
- SlidingWindowAggregationNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitLimit(LimitNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitOffset(OffsetNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitAggregation(AggregationNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitSort(SortNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitProject(ProjectNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitTimeJoin(TimeJoinNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitTransform(TransformNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitLastQueryScan(LastQueryScanNode node, MemoryDistributionContext context) {
- // do nothing since SourceNode will not have Exchange/FragmentSink as child
- return null;
- }
-
- @Override
- public Void visitAlignedLastQueryScan(
- AlignedLastQueryScanNode node, MemoryDistributionContext context) {
- // do nothing since SourceNode will not have Exchange/FragmentSink as child
- return null;
- }
-
- @Override
- public Void visitLastQuery(LastQueryNode node, MemoryDistributionContext context) {
- processConsumeChildrenOneByOneNode(node);
- return null;
- }
-
- @Override
- public Void visitLastQueryMerge(LastQueryMergeNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitLastQueryCollect(LastQueryCollectNode node, MemoryDistributionContext context) {
- processConsumeChildrenOneByOneNode(node);
- return null;
- }
-
- @Override
- public Void visitInto(IntoNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitDeviceViewInto(DeviceViewIntoNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitVerticallyConcat(VerticallyConcatNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitSchemaQueryMerge(SchemaQueryMergeNode node, MemoryDistributionContext context) {
- processConsumeChildrenOneByOneNode(node);
- return null;
- }
-
- @Override
- public Void visitSchemaQueryScan(SchemaQueryScanNode node, MemoryDistributionContext context) {
- // do nothing since SourceNode will not have Exchange/FragmentSink as child
- return null;
- }
-
- @Override
- public Void visitSchemaQueryOrderByHeat(
- SchemaQueryOrderByHeatNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitTimeSeriesSchemaScan(
- TimeSeriesSchemaScanNode node, MemoryDistributionContext context) {
- // do nothing since SourceNode will not have Exchange/FragmentSink as child
- return null;
- }
-
- @Override
- public Void visitDevicesSchemaScan(
- DevicesSchemaScanNode node, MemoryDistributionContext context) {
- // do nothing since SourceNode will not have Exchange/FragmentSink as child
- return null;
- }
-
- @Override
- public Void visitDevicesCount(DevicesCountNode node, MemoryDistributionContext context) {
- // do nothing since SourceNode will not have Exchange/FragmentSink as child
- return null;
- }
-
- @Override
- public Void visitTimeSeriesCount(TimeSeriesCountNode node, MemoryDistributionContext context) {
- // do nothing since SourceNode will not have Exchange/FragmentSink as child
- return null;
- }
-
- @Override
- public Void visitLevelTimeSeriesCount(
- LevelTimeSeriesCountNode node, MemoryDistributionContext context) {
- // do nothing since SourceNode will not have Exchange/FragmentSink as child
- return null;
- }
-
- @Override
- public Void visitCountMerge(CountSchemaMergeNode node, MemoryDistributionContext context) {
- processConsumeChildrenOneByOneNode(node);
- return null;
- }
-
- @Override
- public Void visitSchemaFetchMerge(SchemaFetchMergeNode node, MemoryDistributionContext context) {
- processConsumeChildrenOneByOneNode(node);
- return null;
- }
-
- @Override
- public Void visitSchemaFetchScan(SchemaFetchScanNode node, MemoryDistributionContext context) {
- // do nothing since SourceNode will not have Exchange/FragmentSink as child
- return null;
- }
-
- @Override
- public Void visitNodePathsSchemaScan(
- NodePathsSchemaScanNode node, MemoryDistributionContext context) {
- // do nothing since SourceNode will not have Exchange/FragmentSink as child
- return null;
- }
-
- @Override
- public Void visitNodeManagementMemoryMerge(
- NodeManagementMemoryMergeNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitNodePathConvert(NodePathsConvertNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitNodePathsCount(NodePathsCountNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitConstructSchemaBlackList(
- ConstructSchemaBlackListNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitRollbackSchemaBlackList(
- RollbackSchemaBlackListNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitSingleDeviceView(SingleDeviceViewNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitMergeSort(MergeSortNode node, MemoryDistributionContext context) {
- processConsumeAllChildrenAtTheSameTime(node);
- return null;
- }
-
- @Override
- public Void visitShowQueries(ShowQueriesNode node, MemoryDistributionContext context) {
- // do nothing since VirtualSourceNode will not have Exchange/FragmentSink as child
- return null;
- }
-
- enum MemoryDistributionType {
- /**
- * This type means that this node needs data from all the children. For example, TimeJoinNode.
- * If the type of the father node of an ExchangeNode is CONSUME_ALL_CHILDREN_AT_THE_SAME_TIME,
- * the ExchangeNode needs one split of the memory.
- */
- CONSUME_ALL_CHILDREN_AT_THE_SAME_TIME(0),
-
- /**
- * This type means that this node consumes data of the children one by one. For example,
- * DeviceMergeNode. If the type of the father node of an ExchangeNode is
- * CONSUME_CHILDREN_ONE_BY_ONE, all the ExchangeNodes of that father node share one split of the
- * memory.
- */
- CONSUME_CHILDREN_ONE_BY_ONE(1);
-
- private final int id;
-
- MemoryDistributionType(int id) {
- this.id = id;
- }
-
- public int getId() {
- return id;
- }
- }
-
- static class MemoryDistributionContext {
- final PlanNodeId planNodeId;
- boolean exchangeAdded = false;
- final MemoryDistributionType memoryDistributionType;
-
- MemoryDistributionContext(
- PlanNodeId planNodeId, MemoryDistributionType memoryDistributionType) {
- this.planNodeId = planNodeId;
- this.memoryDistributionType = memoryDistributionType;
- }
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/OperatorTreeGenerator.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/OperatorTreeGenerator.java
index d5aed8eb76..1e41632155 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/OperatorTreeGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/OperatorTreeGenerator.java
@@ -31,11 +31,14 @@ import org.apache.iotdb.db.mpp.aggregation.slidingwindow.SlidingWindowAggregator
import org.apache.iotdb.db.mpp.aggregation.timerangeiterator.ITimeRangeIterator;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.NodeRef;
+import org.apache.iotdb.db.mpp.execution.driver.DataDriverContext;
import org.apache.iotdb.db.mpp.execution.driver.SchemaDriverContext;
import org.apache.iotdb.db.mpp.execution.exchange.ISinkHandle;
import org.apache.iotdb.db.mpp.execution.exchange.ISourceHandle;
+import org.apache.iotdb.db.mpp.execution.exchange.LocalSinkHandle;
import org.apache.iotdb.db.mpp.execution.exchange.MPPDataExchangeManager;
import org.apache.iotdb.db.mpp.execution.exchange.MPPDataExchangeService;
+import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceManager;
import org.apache.iotdb.db.mpp.execution.operator.AggregationUtil;
import org.apache.iotdb.db.mpp.execution.operator.Operator;
import org.apache.iotdb.db.mpp.execution.operator.OperatorContext;
@@ -252,7 +255,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
boolean ascending = node.getScanOrder() == Ordering.ASC;
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -262,17 +265,18 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
Filter valueFilter = node.getValueFilter();
SeriesScanOperator seriesScanOperator =
new SeriesScanOperator(
+ operatorContext,
node.getPlanNodeId(),
seriesPath,
context.getAllSensors(seriesPath.getDevice(), seriesPath.getMeasurement()),
seriesPath.getSeriesType(),
- operatorContext,
timeFilter != null ? timeFilter.copy() : null,
valueFilter != null ? valueFilter.copy() : null,
ascending);
- context.addSourceOperator(seriesScanOperator);
- context.addPath(seriesPath);
+ ((DataDriverContext) context.getDriverContext()).addSourceOperator(seriesScanOperator);
+ ((DataDriverContext) context.getDriverContext()).addPath(seriesPath);
+ context.getDriverContext().setInputDriver(true);
context.getTimeSliceAllocator().recordExecutionWeight(operatorContext, 1);
return seriesScanOperator;
}
@@ -284,7 +288,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
boolean ascending = node.getScanOrder() == Ordering.ASC;
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -301,8 +305,9 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
valueFilter != null ? valueFilter.copy() : null,
ascending);
- context.addSourceOperator(seriesScanOperator);
- context.addPath(seriesPath);
+ ((DataDriverContext) context.getDriverContext()).addSourceOperator(seriesScanOperator);
+ ((DataDriverContext) context.getDriverContext()).addPath(seriesPath);
+ context.getDriverContext().setInputDriver(true);
context
.getTimeSliceAllocator()
.recordExecutionWeight(operatorContext, seriesPath.getColumnNum());
@@ -316,7 +321,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
boolean ascending = node.getScanOrder() == Ordering.ASC;
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -353,8 +358,9 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
node.getGroupByTimeParameter(),
maxReturnSize);
- context.addSourceOperator(aggregateScanOperator);
- context.addPath(seriesPath);
+ ((DataDriverContext) context.getDriverContext()).addSourceOperator(aggregateScanOperator);
+ ((DataDriverContext) context.getDriverContext()).addPath(seriesPath);
+ context.getDriverContext().setInputDriver(true);
context.getTimeSliceAllocator().recordExecutionWeight(operatorContext, aggregators.size());
return aggregateScanOperator;
}
@@ -366,7 +372,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
boolean ascending = node.getScanOrder() == Ordering.ASC;
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -414,8 +420,10 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
groupByTimeParameter,
maxReturnSize);
- context.addSourceOperator(seriesAggregationScanOperator);
- context.addPath(seriesPath);
+ ((DataDriverContext) context.getDriverContext())
+ .addSourceOperator(seriesAggregationScanOperator);
+ ((DataDriverContext) context.getDriverContext()).addPath(seriesPath);
+ context.getDriverContext().setInputDriver(true);
context.getTimeSliceAllocator().recordExecutionWeight(operatorContext, aggregators.size());
return seriesAggregationScanOperator;
}
@@ -428,7 +436,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -464,7 +472,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
TimeSeriesSchemaScanNode node, LocalExecutionPlanContext context) {
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -489,7 +497,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
DevicesSchemaScanNode node, LocalExecutionPlanContext context) {
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -509,11 +517,10 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
@Override
public Operator visitSchemaQueryMerge(
SchemaQueryMergeNode node, LocalExecutionPlanContext context) {
- List<Operator> children =
- node.getChildren().stream().map(n -> n.accept(this, context)).collect(Collectors.toList());
+ List<Operator> children = dealWithConsumeChildrenOneByOneNode(node, context);
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -524,11 +531,10 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
@Override
public Operator visitCountMerge(CountSchemaMergeNode node, LocalExecutionPlanContext context) {
- List<Operator> children =
- node.getChildren().stream().map(n -> n.accept(this, context)).collect(Collectors.toList());
+ List<Operator> children = dealWithConsumeChildrenOneByOneNode(node, context);
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -545,7 +551,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
public Operator visitDevicesCount(DevicesCountNode node, LocalExecutionPlanContext context) {
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -562,7 +568,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
TimeSeriesCountNode node, LocalExecutionPlanContext context) {
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -585,7 +591,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
LevelTimeSeriesCountNode node, LocalExecutionPlanContext context) {
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -609,7 +615,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
NodePathsSchemaScanNode node, LocalExecutionPlanContext context) {
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -627,7 +633,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
Operator child = node.getChild().accept(this, context);
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -642,7 +648,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
Operator child = node.getChild().accept(this, context);
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -656,7 +662,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
Operator child = node.getChild().accept(this, context);
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -670,7 +676,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
SingleDeviceViewNode node, LocalExecutionPlanContext context) {
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -693,15 +699,12 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
public Operator visitDeviceView(DeviceViewNode node, LocalExecutionPlanContext context) {
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
DeviceViewOperator.class.getSimpleName());
- List<Operator> children =
- node.getChildren().stream()
- .map(child -> child.accept(this, context))
- .collect(Collectors.toList());
+ List<Operator> children = dealWithConsumeChildrenOneByOneNode(node, context);
List<List<Integer>> deviceColumnIndex =
node.getDevices().stream()
.map(deviceName -> node.getDeviceToMeasurementIndexesMap().get(deviceName))
@@ -717,15 +720,12 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
public Operator visitDeviceMerge(DeviceMergeNode node, LocalExecutionPlanContext context) {
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
DeviceMergeOperator.class.getSimpleName());
- List<Operator> children =
- node.getChildren().stream()
- .map(child -> child.accept(this, context))
- .collect(Collectors.toList());
+ List<Operator> children = dealWithConsumeAllChildrenPipelineBreaker(node, context);
List<TSDataType> dataTypes = getOutputColumnTypes(node, context.getTypeProvider());
TimeSelector selector = null;
TimeComparator timeComparator = null;
@@ -752,18 +752,14 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
public Operator visitMergeSort(MergeSortNode node, LocalExecutionPlanContext context) {
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
MergeSortOperator.class.getSimpleName());
List<TSDataType> dataTypes = getOutputColumnTypes(node, context.getTypeProvider());
context.setCachedDataTypes(dataTypes);
- List<Operator> children =
- node.getChildren().stream()
- .map(child -> child.accept(this, context))
- .collect(Collectors.toList());
-
+ List<Operator> children = dealWithConsumeAllChildrenPipelineBreaker(node, context);
List<SortItem> sortItemList = node.getMergeOrderParameter().getSortItemList();
context.getTimeSliceAllocator().recordExecutionWeight(operatorContext, 1);
@@ -820,7 +816,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
FillPolicy fillPolicy = descriptor.getFillPolicy();
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -938,7 +934,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
public Operator visitTransform(TransformNode node, LocalExecutionPlanContext context) {
final OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1040,7 +1036,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
final List<TSDataType> filterOutputDataTypes = new ArrayList<>(inputDataTypes);
final OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1142,7 +1138,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
try {
final OperatorContext transformContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1168,10 +1164,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
checkArgument(
!node.getGroupByLevelDescriptors().isEmpty(),
"GroupByLevel descriptorList cannot be empty");
- List<Operator> children =
- node.getChildren().stream()
- .map(child -> child.accept(this, context))
- .collect(Collectors.toList());
+ List<Operator> children = dealWithConsumeAllChildrenPipelineBreaker(node, context);
boolean ascending = node.getScanOrder() == Ordering.ASC;
List<Aggregator> aggregators = new ArrayList<>();
Map<String, List<InputLocation>> layout = makeLayout(node);
@@ -1193,7 +1186,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
}
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1218,10 +1211,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
node.getTagValuesToAggregationDescriptors().size() >= 1,
"GroupByTag aggregation descriptors cannot be empty");
- List<Operator> children =
- node.getChildren().stream()
- .map(child -> child.accept(this, context))
- .collect(Collectors.toList());
+ List<Operator> children = dealWithConsumeAllChildrenPipelineBreaker(node, context);
boolean ascending = node.getScanOrder() == Ordering.ASC;
Map<String, List<InputLocation>> layout = makeLayout(node);
@@ -1264,7 +1254,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
aggregationDescriptors, timeRangeIterator, context.getTypeProvider());
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1282,7 +1272,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
"Aggregation descriptorList cannot be empty");
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1329,7 +1319,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
Operator child = node.getChild().accept(this, context);
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1344,7 +1334,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
Operator child = node.getChild().accept(this, context);
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1359,10 +1349,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
checkArgument(
!node.getAggregationDescriptorList().isEmpty(),
"Aggregation descriptorList cannot be empty");
- List<Operator> children =
- node.getChildren().stream()
- .map(child -> child.accept(this, context))
- .collect(Collectors.toList());
+ List<Operator> children = dealWithConsumeAllChildrenPipelineBreaker(node, context);
boolean ascending = node.getScanOrder() == Ordering.ASC;
List<Aggregator> aggregators = new ArrayList<>();
Map<String, List<InputLocation>> layout = makeLayout(node);
@@ -1388,7 +1375,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
checkArgument(children.size() == 1, "rawDataAggregateOperator can only accept one input");
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1411,7 +1398,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
} else {
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1457,7 +1444,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
Operator child = node.getChild().accept(this, context);
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1487,7 +1474,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
Operator child = node.getChild().accept(this, context);
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1521,7 +1508,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
intoPathDescriptor.getTargetDeviceToAlignedMap(),
intoPathDescriptor.getSourceTargetPathPairList(),
sourceColumnToInputLocationMap,
- context.getIntoOperationExecutor(),
+ FragmentInstanceManager.getInstance().getIntoOperationExecutor(),
maxStatementSize);
}
@@ -1530,7 +1517,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
Operator child = node.getChild().accept(this, context);
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1579,7 +1566,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
deviceViewIntoPathDescriptor.getTargetDeviceToAlignedMap(),
deviceViewIntoPathDescriptor.getDeviceToSourceTargetPathPairListMap(),
sourceColumnToInputLocationMap,
- context.getIntoOperationExecutor(),
+ FragmentInstanceManager.getInstance().getIntoOperationExecutor(),
maxStatementSize);
}
@@ -1648,13 +1635,10 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
@Override
public Operator visitTimeJoin(TimeJoinNode node, LocalExecutionPlanContext context) {
- List<Operator> children =
- node.getChildren().stream()
- .map(child -> child.accept(this, context))
- .collect(Collectors.toList());
+ List<Operator> children = dealWithConsumeAllChildrenPipelineBreaker(node, context);
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1678,13 +1662,10 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
@Override
public Operator visitVerticallyConcat(
VerticallyConcatNode node, LocalExecutionPlanContext context) {
- List<Operator> children =
- node.getChildren().stream()
- .map(child -> child.accept(this, context))
- .collect(Collectors.toList());
+ List<Operator> children = dealWithConsumeAllChildrenPipelineBreaker(node, context);
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1699,7 +1680,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
public Operator visitShowQueries(ShowQueriesNode node, LocalExecutionPlanContext context) {
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1743,7 +1724,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
public Operator visitExchange(ExchangeNode node, LocalExecutionPlanContext context) {
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1756,7 +1737,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
TEndPoint upstreamEndPoint = node.getUpstreamEndpoint();
ISourceHandle sourceHandle =
isSameNode(upstreamEndPoint)
- ? MPP_DATA_EXCHANGE_MANAGER.createLocalSourceHandle(
+ ? MPP_DATA_EXCHANGE_MANAGER.createLocalSourceHandleForFragment(
localInstanceId.toThrift(),
node.getPlanNodeId().getId(),
remoteInstanceId.toThrift(),
@@ -1767,11 +1748,19 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
upstreamEndPoint,
remoteInstanceId.toThrift(),
context.getInstanceContext()::failed);
- return new ExchangeOperator(operatorContext, sourceHandle, node.getUpstreamPlanNodeId());
+ context.addExchangeSumNum(1);
+ sourceHandle.setMaxBytesCanReserve(context.getMaxBytesOneHandleCanReserve());
+ ExchangeOperator exchangeOperator =
+ new ExchangeOperator(operatorContext, sourceHandle, node.getUpstreamPlanNodeId());
+ context.addExchangeOperator(exchangeOperator);
+ return exchangeOperator;
}
@Override
public Operator visitFragmentSink(FragmentSinkNode node, LocalExecutionPlanContext context) {
+ if (!isSameNode(node.getDownStreamEndpoint())) {
+ context.addExchangeSumNum(1);
+ }
Operator child = node.getChild().accept(this, context);
FragmentInstanceId localInstanceId = context.getInstanceContext().getId();
@@ -1783,7 +1772,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
ISinkHandle sinkHandle =
isSameNode(downStreamEndPoint)
- ? MPP_DATA_EXCHANGE_MANAGER.createLocalSinkHandle(
+ ? MPP_DATA_EXCHANGE_MANAGER.createLocalSinkHandleForFragment(
localInstanceId.toThrift(),
targetInstanceId.toThrift(),
node.getDownStreamPlanNodeId().getId(),
@@ -1795,18 +1784,18 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
node.getDownStreamPlanNodeId().getId(),
node.getPlanNodeId().getId(),
context.getInstanceContext());
- context.setSinkHandle(sinkHandle);
+ sinkHandle.setMaxBytesCanReserve(context.getMaxBytesOneHandleCanReserve());
+ context.getDriverContext().setSinkHandle(sinkHandle);
return child;
}
@Override
public Operator visitSchemaFetchMerge(
SchemaFetchMergeNode node, LocalExecutionPlanContext context) {
- List<Operator> children =
- node.getChildren().stream().map(n -> n.accept(this, context)).collect(Collectors.toList());
+ List<Operator> children = dealWithConsumeChildrenOneByOneNode(node, context);
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1820,7 +1809,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
SchemaFetchScanNode node, LocalExecutionPlanContext context) {
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1831,7 +1820,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
operatorContext,
node.getPatternTree(),
node.getTemplateMap(),
- ((SchemaDriverContext) (context.getInstanceContext().getDriverContext())).getSchemaRegion(),
+ ((SchemaDriverContext) (context.getDriverContext())).getSchemaRegion(),
node.isWithTags());
}
@@ -1866,7 +1855,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1886,7 +1875,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
MeasurementPath seriesPath = node.getSeriesPath();
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1911,8 +1900,9 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
false,
null,
maxReturnSize);
- context.addSourceOperator(seriesAggregationScanOperator);
- context.addPath(seriesPath);
+ ((DataDriverContext) context.getDriverContext())
+ .addSourceOperator(seriesAggregationScanOperator);
+ ((DataDriverContext) context.getDriverContext()).addPath(seriesPath);
context.getTimeSliceAllocator().recordExecutionWeight(operatorContext, aggregators.size());
return seriesAggregationScanOperator;
}
@@ -1963,7 +1953,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -1981,7 +1971,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
AlignedLastQueryScanNode node, AlignedPath unCachedPath, LocalExecutionPlanContext context) {
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -2007,8 +1997,9 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
false,
null,
maxReturnSize);
- context.addSourceOperator(seriesAggregationScanOperator);
- context.addPath(unCachedPath);
+ ((DataDriverContext) context.getDriverContext())
+ .addSourceOperator(seriesAggregationScanOperator);
+ ((DataDriverContext) context.getDriverContext()).addPath(unCachedPath);
context.getTimeSliceAllocator().recordExecutionWeight(operatorContext, aggregators.size());
return seriesAggregationScanOperator;
}
@@ -2050,7 +2041,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
}
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -2081,7 +2072,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -2099,7 +2090,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
.collect(Collectors.toList());
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -2118,13 +2109,10 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
@Override
public Operator visitLastQueryCollect(
LastQueryCollectNode node, LocalExecutionPlanContext context) {
- List<Operator> children =
- node.getChildren().stream()
- .map(child -> child.accept(this, context))
- .collect(Collectors.toList());
+ List<Operator> children = dealWithConsumeChildrenOneByOneNode(node, context);
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -2177,7 +2165,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
PathsUsingTemplateScanNode node, LocalExecutionPlanContext context) {
OperatorContext operatorContext =
context
- .getInstanceContext()
+ .getDriverContext()
.addOperatorContext(
context.getNextOperatorId(),
node.getPlanNodeId(),
@@ -2189,4 +2177,61 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
SchemaSourceFactory.getPathsUsingTemplateSource(
node.getPathPatternList(), node.getTemplateId()));
}
+
+ private List<Operator> dealWithConsumeAllChildrenPipelineBreaker(
+ PlanNode node, LocalExecutionPlanContext context) {
+ // children after pipelining
+ List<Operator> children = new ArrayList<>();
+ int finalExchangeNum = context.getExchangeSumNum();
+ for (PlanNode childSource : node.getChildren()) {
+ // Create pipelines for children
+ LocalExecutionPlanContext subContext = context.createSubContext();
+ Operator childOperation = childSource.accept(this, subContext);
+ // If the child belongs to another fragment instance, we don't create pipeline for it
+ if (childOperation instanceof ExchangeOperator) {
+ children.add(childOperation);
+ finalExchangeNum += 1;
+ } else {
+ ISinkHandle localSinkHandle =
+ MPP_DATA_EXCHANGE_MANAGER.createLocalSinkHandleForPipeline(
+ subContext.getDriverContext(), childSource.getPlanNodeId().getId());
+ subContext.setSinkHandle(localSinkHandle);
+ subContext.addPipelineDriverFactory(childOperation, subContext.getDriverContext());
+
+ ExchangeOperator sourceOperator =
+ new ExchangeOperator(
+ context
+ .getDriverContext()
+ .addOperatorContext(
+ context.getNextOperatorId(), null, ExchangeOperator.class.getSimpleName()),
+ MPP_DATA_EXCHANGE_MANAGER.createLocalSourceHandleForPipeline(
+ ((LocalSinkHandle) localSinkHandle).getSharedTsBlockQueue(),
+ context.getDriverContext()),
+ childSource.getPlanNodeId());
+ context
+ .getTimeSliceAllocator()
+ .recordExecutionWeight(sourceOperator.getOperatorContext(), 1);
+ children.add(sourceOperator);
+ context.addExchangeOperator(sourceOperator);
+ finalExchangeNum += subContext.getExchangeSumNum() - context.getExchangeSumNum() + 1;
+ }
+ }
+ context.setExchangeSumNum(finalExchangeNum);
+ return children;
+ }
+
+ private List<Operator> dealWithConsumeChildrenOneByOneNode(
+ PlanNode node, LocalExecutionPlanContext context) {
+ int originExchangeNum = context.getExchangeSumNum();
+ int finalExchangeNum = context.getExchangeSumNum();
+ List<Operator> children = new ArrayList<>();
+ for (PlanNode childSource : node.getChildren()) {
+ Operator childOperation = childSource.accept(this, context);
+ finalExchangeNum = Math.max(finalExchangeNum, context.getExchangeSumNum());
+ context.setExchangeSumNum(originExchangeNum);
+ children.add(childOperation);
+ }
+ context.setExchangeSumNum(finalExchangeNum);
+ return children;
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/PipelineDriverFactory.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/PipelineDriverFactory.java
new file mode 100644
index 0000000000..ec5eb7a3e2
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/PipelineDriverFactory.java
@@ -0,0 +1,59 @@
+/*
+ * 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.plan.planner;
+
+import org.apache.iotdb.db.mpp.execution.driver.DataDriver;
+import org.apache.iotdb.db.mpp.execution.driver.Driver;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
+import org.apache.iotdb.db.mpp.execution.operator.Operator;
+
+import static java.util.Objects.requireNonNull;
+
+public class PipelineDriverFactory {
+
+ private final DriverContext driverContext;
+ // TODO Use OperatorFactory to replace operator to generate multiple drivers for on pipeline
+ private final Operator operation;
+
+ public PipelineDriverFactory(Operator operation, DriverContext driverContext) {
+ this.operation = requireNonNull(operation, "rootOperator is null");
+ this.driverContext = driverContext;
+ }
+
+ public DriverContext getDriverContext() {
+ return driverContext;
+ }
+
+ public Driver createDriver() {
+ requireNonNull(driverContext, "driverContext is null");
+ try {
+ return new DataDriver(operation, driverContext);
+ } catch (Throwable failure) {
+ try {
+ operation.close();
+ } catch (Throwable closeFailure) {
+ if (failure != closeFailure) {
+ failure.addSuppressed(closeFailure);
+ }
+ }
+ throw failure;
+ }
+ }
+}
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/DataDriverTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/DataDriverTest.java
index 1ce0574de7..2a445636aa 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/DataDriverTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/DataDriverTest.java
@@ -22,7 +22,6 @@ import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.MeasurementPath;
-import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
import org.apache.iotdb.db.engine.storagegroup.DataRegion;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
@@ -52,7 +51,6 @@ import org.apache.iotdb.tsfile.read.common.block.TsBlock;
import org.apache.iotdb.tsfile.read.common.block.column.IntColumn;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
-import com.google.common.collect.ImmutableList;
import com.google.common.util.concurrent.ListenableFuture;
import io.airlift.units.Duration;
import org.junit.After;
@@ -111,29 +109,30 @@ public class DataDriverTest {
new FragmentInstanceId(new PlanFragmentId(queryId, 0), "stub-instance");
FragmentInstanceStateMachine stateMachine =
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
+ DataRegion dataRegion = Mockito.mock(DataRegion.class);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ fragmentInstanceContext.setDataRegion(dataRegion);
+ DataDriverContext driverContext = new DataDriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(
3, new PlanNodeId("3"), TimeJoinOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
- 4, new PlanNodeId("4"), LimitOperator.class.getSimpleName());
+ driverContext.addOperatorContext(4, new PlanNodeId("4"), LimitOperator.class.getSimpleName());
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
true);
-
+ driverContext.addSourceOperator(seriesScanOperator1);
+ driverContext.addPath(measurementPath1);
seriesScanOperator1
.getOperatorContext()
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
@@ -142,14 +141,16 @@ public class DataDriverTest {
new MeasurementPath(DATA_DRIVER_TEST_SG + ".device0.sensor1", TSDataType.INT32);
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath2,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
true);
+ driverContext.addSourceOperator(seriesScanOperator2);
+ driverContext.addPath(measurementPath2);
seriesScanOperator2
.getOperatorContext()
@@ -157,7 +158,7 @@ public class DataDriverTest {
TimeJoinOperator timeJoinOperator =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
Arrays.asList(seriesScanOperator1, seriesScanOperator2),
Ordering.ASC,
Arrays.asList(TSDataType.INT32, TSDataType.INT32),
@@ -167,30 +168,21 @@ public class DataDriverTest {
new AscTimeComparator());
LimitOperator limitOperator =
- new LimitOperator(
- fragmentInstanceContext.getOperatorContexts().get(3), 250, timeJoinOperator);
-
- DataRegion dataRegion = Mockito.mock(DataRegion.class);
+ new LimitOperator(driverContext.getOperatorContexts().get(3), 250, timeJoinOperator);
- List<PartialPath> pathList = ImmutableList.of(measurementPath1, measurementPath2);
String deviceId = DATA_DRIVER_TEST_SG + ".device0";
-
- Mockito.when(dataRegion.query(pathList, deviceId, fragmentInstanceContext, null))
+ Mockito.when(
+ dataRegion.query(driverContext.getPaths(), deviceId, fragmentInstanceContext, null))
.thenReturn(new QueryDataSource(seqResources, unSeqResources));
-
- DataDriverContext driverContext =
- new DataDriverContext(
- fragmentInstanceContext,
- pathList,
- null,
- dataRegion,
- ImmutableList.of(seriesScanOperator1, seriesScanOperator2));
+ fragmentInstanceContext.initQueryDataSource(driverContext.getPaths());
StubSinkHandle sinkHandle = new StubSinkHandle(fragmentInstanceContext);
+ driverContext.setSinkHandle(sinkHandle);
IDriver dataDriver = null;
try {
- dataDriver = new DataDriver(limitOperator, sinkHandle, driverContext);
- assertEquals(fragmentInstanceContext.getId(), dataDriver.getInfo());
+ dataDriver = new DataDriver(limitOperator, driverContext);
+ assertEquals(
+ fragmentInstanceContext.getId(), dataDriver.getDriverTaskId().getFragmentInstanceId());
assertFalse(dataDriver.isFinished());
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/exchange/LocalSinkHandleTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/exchange/LocalSinkHandleTest.java
index 37055a6a24..0a022ae06b 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/exchange/LocalSinkHandleTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/exchange/LocalSinkHandleTest.java
@@ -50,25 +50,20 @@ public class LocalSinkHandleTest {
SharedTsBlockQueue queue =
new SharedTsBlockQueue(remoteFragmentInstanceId, remotePlanNodeId, mockLocalMemoryManager);
+ // Construct SinkHandle.
+ LocalSinkHandle localSinkHandle =
+ new LocalSinkHandle(localFragmentInstanceId, queue, mockSinkHandleListener);
+
queue.setMaxBytesCanReserve(Long.MAX_VALUE);
// Construct SourceHandle
LocalSourceHandle localSourceHandle =
new LocalSourceHandle(
localFragmentInstanceId,
- remoteFragmentInstanceId,
remotePlanNodeId,
queue,
Mockito.mock(MPPDataExchangeManager.SourceHandleListener.class));
- // Construct SinkHandle.
- LocalSinkHandle localSinkHandle =
- new LocalSinkHandle(
- remoteFragmentInstanceId,
- remotePlanNodeId,
- localFragmentInstanceId,
- queue,
- mockSinkHandleListener);
Assert.assertFalse(localSinkHandle.isFull().isDone());
localSourceHandle.isBlocked();
// blocked of LocalSinkHandle should be completed after calling isBlocked() of corresponding
@@ -137,25 +132,20 @@ public class LocalSinkHandleTest {
SharedTsBlockQueue queue =
new SharedTsBlockQueue(remoteFragmentInstanceId, remotePlanNodeId, mockLocalMemoryManager);
+ // Construct SinkHandle.
+ LocalSinkHandle localSinkHandle =
+ new LocalSinkHandle(localFragmentInstanceId, queue, mockSinkHandleListener);
+
queue.setMaxBytesCanReserve(Long.MAX_VALUE);
// Construct SourceHandle
LocalSourceHandle localSourceHandle =
new LocalSourceHandle(
localFragmentInstanceId,
- remoteFragmentInstanceId,
remotePlanNodeId,
queue,
Mockito.mock(MPPDataExchangeManager.SourceHandleListener.class));
- // Construct SinkHandle.
- LocalSinkHandle localSinkHandle =
- new LocalSinkHandle(
- remoteFragmentInstanceId,
- remotePlanNodeId,
- localFragmentInstanceId,
- queue,
- mockSinkHandleListener);
Assert.assertFalse(localSinkHandle.isFull().isDone());
localSourceHandle.isBlocked();
// blocked of LocalSinkHandle should be completed after calling isBlocked() of corresponding
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/exchange/LocalSourceHandleTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/exchange/LocalSourceHandleTest.java
index 3c9ee13e39..674a80ce3e 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/exchange/LocalSourceHandleTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/exchange/LocalSourceHandleTest.java
@@ -50,11 +50,7 @@ public class LocalSourceHandleTest {
LocalSourceHandle localSourceHandle =
new LocalSourceHandle(
- remoteFragmentInstanceId,
- localFragmentInstanceId,
- localPlanNodeId,
- queue,
- mockSourceHandleListener);
+ localFragmentInstanceId, localPlanNodeId, queue, mockSourceHandleListener);
Assert.assertFalse(localSourceHandle.isBlocked().isDone());
Assert.assertFalse(localSourceHandle.isAborted());
Assert.assertFalse(localSourceHandle.isFinished());
@@ -98,11 +94,7 @@ public class LocalSourceHandleTest {
LocalSourceHandle localSourceHandle =
new LocalSourceHandle(
- remoteFragmentInstanceId,
- localFragmentInstanceId,
- localPlanNodeId,
- queue,
- mockSourceHandleListener);
+ localFragmentInstanceId, localPlanNodeId, queue, mockSourceHandleListener);
ListenableFuture<?> future = localSourceHandle.isBlocked();
Assert.assertFalse(future.isDone());
Assert.assertFalse(localSourceHandle.isAborted());
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/exchange/MPPDataExchangeManagerTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/exchange/MPPDataExchangeManagerTest.java
index add9d54009..786d5b1830 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/exchange/MPPDataExchangeManagerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/exchange/MPPDataExchangeManagerTest.java
@@ -58,7 +58,7 @@ public class MPPDataExchangeManagerTest {
new ClientPoolFactory.SyncDataNodeMPPDataExchangeServiceClientPoolFactory()));
ISinkHandle localSinkHandle =
- mppDataExchangeManager.createLocalSinkHandle(
+ mppDataExchangeManager.createLocalSinkHandleForFragment(
localFragmentInstanceId,
remoteFragmentInstanceId,
remotePlanNodeId,
@@ -67,7 +67,7 @@ public class MPPDataExchangeManagerTest {
Assert.assertTrue(localSinkHandle instanceof LocalSinkHandle);
ISourceHandle localSourceHandle =
- mppDataExchangeManager.createLocalSourceHandle(
+ mppDataExchangeManager.createLocalSourceHandleForFragment(
remoteFragmentInstanceId, remotePlanNodeId, localFragmentInstanceId, t -> {});
Assert.assertTrue(localSourceHandle instanceof LocalSourceHandle);
@@ -100,13 +100,13 @@ public class MPPDataExchangeManagerTest {
new ClientPoolFactory.SyncDataNodeMPPDataExchangeServiceClientPoolFactory()));
ISourceHandle localSourceHandle =
- mppDataExchangeManager.createLocalSourceHandle(
+ mppDataExchangeManager.createLocalSourceHandleForFragment(
localFragmentInstanceId, localPlanNodeId, remoteFragmentInstanceId, t -> {});
Assert.assertTrue(localSourceHandle instanceof LocalSourceHandle);
ISinkHandle localSinkHandle =
- mppDataExchangeManager.createLocalSinkHandle(
+ mppDataExchangeManager.createLocalSinkHandleForFragment(
remoteFragmentInstanceId,
localFragmentInstanceId,
localPlanNodeId,
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/AggregationOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/AggregationOperatorTest.java
index 2fb6c8909a..28fb400e1d 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/AggregationOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/AggregationOperatorTest.java
@@ -31,6 +31,7 @@ import org.apache.iotdb.db.mpp.aggregation.Aggregator;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.AggregationOperator;
@@ -293,16 +294,16 @@ public class AggregationOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId1, SeriesAggregationScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
2, planNodeId2, SeriesAggregationScanOperator.class.getSimpleName());
PlanNodeId planNodeId3 = new PlanNodeId("3");
- fragmentInstanceContext.addOperatorContext(
- 3, planNodeId3, AggregationOperator.class.getSimpleName());
- fragmentInstanceContext
+ driverContext.addOperatorContext(3, planNodeId3, AggregationOperator.class.getSimpleName());
+ driverContext
.getOperatorContexts()
.forEach(
operatorContext -> {
@@ -319,7 +320,7 @@ public class AggregationOperatorTest {
planNodeId1,
measurementPath1,
Collections.singleton("sensor0"),
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
aggregators,
initTimeRangeIterator(groupByTimeParameter, true, true),
null,
@@ -343,7 +344,7 @@ public class AggregationOperatorTest {
planNodeId2,
measurementPath1,
Collections.singleton("sensor0"),
- fragmentInstanceContext.getOperatorContexts().get(1),
+ driverContext.getOperatorContexts().get(1),
aggregators,
initTimeRangeIterator(groupByTimeParameter, true, true),
null,
@@ -372,7 +373,7 @@ public class AggregationOperatorTest {
}
return new AggregationOperator(
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
finalAggregators,
initTimeRangeIterator(groupByTimeParameter, true, true),
children,
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/AlignedSeriesAggregationScanOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/AlignedSeriesAggregationScanOperatorTest.java
index 269e9abcbf..699e818c3e 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/AlignedSeriesAggregationScanOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/AlignedSeriesAggregationScanOperatorTest.java
@@ -30,6 +30,7 @@ import org.apache.iotdb.db.mpp.aggregation.Aggregator;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.source.AlignedSeriesAggregationScanOperator;
@@ -618,10 +619,10 @@ public class AlignedSeriesAggregationScanOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext
+ driverContext.addOperatorContext(1, planNodeId, SeriesScanOperator.class.getSimpleName());
+ driverContext
.getOperatorContexts()
.forEach(operatorContext -> operatorContext.setMaxRunTime(TEST_TIME_SLICE));
@@ -629,7 +630,7 @@ public class AlignedSeriesAggregationScanOperatorTest {
new AlignedSeriesAggregationScanOperator(
planNodeId,
alignedPath,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
aggregators,
initTimeRangeIterator(groupByTimeParameter, ascending, true),
timeFilter,
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/AlignedSeriesScanOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/AlignedSeriesScanOperatorTest.java
index f2eefccea3..77b1de3a23 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/AlignedSeriesScanOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/AlignedSeriesScanOperatorTest.java
@@ -28,6 +28,7 @@ import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.join.TimeJoinOperator;
@@ -113,15 +114,16 @@ public class AlignedSeriesScanOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId, AlignedSeriesScanOperator.class.getSimpleName());
AlignedSeriesScanOperator seriesScanOperator =
new AlignedSeriesScanOperator(
planNodeId,
alignedPath,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
null,
null,
true);
@@ -190,37 +192,32 @@ public class AlignedSeriesScanOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId1, AlignedSeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
2, planNodeId2, AlignedSeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId3 = new PlanNodeId("3");
- fragmentInstanceContext.addOperatorContext(
- 3, planNodeId3, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(3, planNodeId3, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId4 = new PlanNodeId("4");
- fragmentInstanceContext.addOperatorContext(
- 4, planNodeId4, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(4, planNodeId4, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId5 = new PlanNodeId("5");
- fragmentInstanceContext.addOperatorContext(
- 5, planNodeId5, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(5, planNodeId5, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId6 = new PlanNodeId("6");
- fragmentInstanceContext.addOperatorContext(
- 6, planNodeId6, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(6, planNodeId6, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId7 = new PlanNodeId("7");
- fragmentInstanceContext.addOperatorContext(
- 7, planNodeId7, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(7, planNodeId7, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId8 = new PlanNodeId("8");
- fragmentInstanceContext.addOperatorContext(
- 8, planNodeId8, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(8, planNodeId8, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(
9, new PlanNodeId("9"), TimeJoinOperator.class.getSimpleName());
AlignedSeriesScanOperator seriesScanOperator1 =
new AlignedSeriesScanOperator(
planNodeId1,
alignedPath1,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
null,
null,
true);
@@ -242,7 +239,7 @@ public class AlignedSeriesScanOperatorTest {
new AlignedSeriesScanOperator(
planNodeId2,
alignedPath2,
- fragmentInstanceContext.getOperatorContexts().get(1),
+ driverContext.getOperatorContexts().get(1),
null,
null,
true);
@@ -264,11 +261,11 @@ public class AlignedSeriesScanOperatorTest {
SERIES_SCAN_OPERATOR_TEST_SG + ".device2.sensor0", TSDataType.BOOLEAN);
SeriesScanOperator seriesScanOperator3 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(2),
planNodeId3,
measurementPath3,
allSensors,
TSDataType.BOOLEAN,
- fragmentInstanceContext.getOperatorContexts().get(2),
null,
null,
true);
@@ -281,11 +278,11 @@ public class AlignedSeriesScanOperatorTest {
new MeasurementPath(SERIES_SCAN_OPERATOR_TEST_SG + ".device2.sensor1", TSDataType.INT32);
SeriesScanOperator seriesScanOperator4 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(3),
planNodeId4,
measurementPath4,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(3),
null,
null,
true);
@@ -298,11 +295,11 @@ public class AlignedSeriesScanOperatorTest {
new MeasurementPath(SERIES_SCAN_OPERATOR_TEST_SG + ".device2.sensor2", TSDataType.INT64);
SeriesScanOperator seriesScanOperator5 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(4),
planNodeId5,
measurementPath5,
allSensors,
TSDataType.INT64,
- fragmentInstanceContext.getOperatorContexts().get(4),
null,
null,
true);
@@ -315,11 +312,11 @@ public class AlignedSeriesScanOperatorTest {
new MeasurementPath(SERIES_SCAN_OPERATOR_TEST_SG + ".device2.sensor3", TSDataType.FLOAT);
SeriesScanOperator seriesScanOperator6 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(5),
planNodeId6,
measurementPath6,
allSensors,
TSDataType.FLOAT,
- fragmentInstanceContext.getOperatorContexts().get(5),
null,
null,
true);
@@ -332,11 +329,11 @@ public class AlignedSeriesScanOperatorTest {
new MeasurementPath(SERIES_SCAN_OPERATOR_TEST_SG + ".device2.sensor4", TSDataType.DOUBLE);
SeriesScanOperator seriesScanOperator7 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(6),
planNodeId7,
measurementPath7,
allSensors,
TSDataType.DOUBLE,
- fragmentInstanceContext.getOperatorContexts().get(6),
null,
null,
true);
@@ -349,11 +346,11 @@ public class AlignedSeriesScanOperatorTest {
new MeasurementPath(SERIES_SCAN_OPERATOR_TEST_SG + ".device2.sensor5", TSDataType.DOUBLE);
SeriesScanOperator seriesScanOperator8 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(7),
planNodeId8,
measurementPath8,
allSensors,
TSDataType.TEXT,
- fragmentInstanceContext.getOperatorContexts().get(7),
null,
null,
true);
@@ -364,7 +361,7 @@ public class AlignedSeriesScanOperatorTest {
TimeJoinOperator timeJoinOperator =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(8),
+ driverContext.getOperatorContexts().get(8),
Arrays.asList(
seriesScanOperator1,
seriesScanOperator2,
@@ -501,37 +498,32 @@ public class AlignedSeriesScanOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId1, AlignedSeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
2, planNodeId2, AlignedSeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId3 = new PlanNodeId("3");
- fragmentInstanceContext.addOperatorContext(
- 3, planNodeId3, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(3, planNodeId3, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId4 = new PlanNodeId("4");
- fragmentInstanceContext.addOperatorContext(
- 4, planNodeId4, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(4, planNodeId4, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId5 = new PlanNodeId("5");
- fragmentInstanceContext.addOperatorContext(
- 5, planNodeId5, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(5, planNodeId5, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId6 = new PlanNodeId("6");
- fragmentInstanceContext.addOperatorContext(
- 6, planNodeId6, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(6, planNodeId6, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId7 = new PlanNodeId("7");
- fragmentInstanceContext.addOperatorContext(
- 7, planNodeId7, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(7, planNodeId7, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId8 = new PlanNodeId("8");
- fragmentInstanceContext.addOperatorContext(
- 8, planNodeId8, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(8, planNodeId8, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(
9, new PlanNodeId("9"), TimeJoinOperator.class.getSimpleName());
AlignedSeriesScanOperator seriesScanOperator1 =
new AlignedSeriesScanOperator(
planNodeId1,
alignedPath1,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
null,
null,
false);
@@ -553,7 +545,7 @@ public class AlignedSeriesScanOperatorTest {
new AlignedSeriesScanOperator(
planNodeId2,
alignedPath2,
- fragmentInstanceContext.getOperatorContexts().get(1),
+ driverContext.getOperatorContexts().get(1),
null,
null,
false);
@@ -575,11 +567,11 @@ public class AlignedSeriesScanOperatorTest {
SERIES_SCAN_OPERATOR_TEST_SG + ".device2.sensor0", TSDataType.BOOLEAN);
SeriesScanOperator seriesScanOperator3 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(2),
planNodeId3,
measurementPath3,
allSensors,
TSDataType.BOOLEAN,
- fragmentInstanceContext.getOperatorContexts().get(2),
null,
null,
false);
@@ -592,11 +584,11 @@ public class AlignedSeriesScanOperatorTest {
new MeasurementPath(SERIES_SCAN_OPERATOR_TEST_SG + ".device2.sensor1", TSDataType.INT32);
SeriesScanOperator seriesScanOperator4 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(3),
planNodeId4,
measurementPath4,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(3),
null,
null,
false);
@@ -609,11 +601,11 @@ public class AlignedSeriesScanOperatorTest {
new MeasurementPath(SERIES_SCAN_OPERATOR_TEST_SG + ".device2.sensor2", TSDataType.INT64);
SeriesScanOperator seriesScanOperator5 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(4),
planNodeId5,
measurementPath5,
allSensors,
TSDataType.INT64,
- fragmentInstanceContext.getOperatorContexts().get(4),
null,
null,
false);
@@ -626,11 +618,11 @@ public class AlignedSeriesScanOperatorTest {
new MeasurementPath(SERIES_SCAN_OPERATOR_TEST_SG + ".device2.sensor3", TSDataType.FLOAT);
SeriesScanOperator seriesScanOperator6 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(5),
planNodeId6,
measurementPath6,
allSensors,
TSDataType.FLOAT,
- fragmentInstanceContext.getOperatorContexts().get(5),
null,
null,
false);
@@ -643,11 +635,11 @@ public class AlignedSeriesScanOperatorTest {
new MeasurementPath(SERIES_SCAN_OPERATOR_TEST_SG + ".device2.sensor4", TSDataType.DOUBLE);
SeriesScanOperator seriesScanOperator7 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(6),
planNodeId7,
measurementPath7,
allSensors,
TSDataType.DOUBLE,
- fragmentInstanceContext.getOperatorContexts().get(6),
null,
null,
false);
@@ -660,11 +652,11 @@ public class AlignedSeriesScanOperatorTest {
new MeasurementPath(SERIES_SCAN_OPERATOR_TEST_SG + ".device2.sensor5", TSDataType.DOUBLE);
SeriesScanOperator seriesScanOperator8 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(7),
planNodeId8,
measurementPath8,
allSensors,
TSDataType.TEXT,
- fragmentInstanceContext.getOperatorContexts().get(7),
null,
null,
false);
@@ -675,7 +667,7 @@ public class AlignedSeriesScanOperatorTest {
TimeJoinOperator timeJoinOperator =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(8),
+ driverContext.getOperatorContexts().get(8),
Arrays.asList(
seriesScanOperator1,
seriesScanOperator2,
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/DeviceMergeOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/DeviceMergeOperatorTest.java
index beb106d4aa..a60c9fe122 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/DeviceMergeOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/DeviceMergeOperatorTest.java
@@ -28,6 +28,7 @@ import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.DeviceMergeOperator;
@@ -102,21 +103,19 @@ public class DeviceMergeOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId3 = new PlanNodeId("3");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
3, planNodeId3, DeviceViewOperatorTest.class.getSimpleName());
PlanNodeId planNodeId4 = new PlanNodeId("4");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
4, planNodeId4, DeviceViewOperatorTest.class.getSimpleName());
PlanNodeId planNodeId5 = new PlanNodeId("5");
- fragmentInstanceContext.addOperatorContext(
- 5, planNodeId5, DeviceMergeOperator.class.getSimpleName());
+ driverContext.addOperatorContext(5, planNodeId5, DeviceMergeOperator.class.getSimpleName());
List<TSDataType> dataTypes = new ArrayList<>();
dataTypes.add(TSDataType.TEXT);
@@ -126,11 +125,11 @@ public class DeviceMergeOperatorTest {
new MeasurementPath(DEVICE_MERGE_OPERATOR_TEST_SG + ".device0.sensor0", TSDataType.INT32);
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
true);
@@ -141,7 +140,7 @@ public class DeviceMergeOperatorTest {
DeviceViewOperator deviceViewOperator1 =
new DeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
Collections.singletonList(DEVICE_MERGE_OPERATOR_TEST_SG + ".device0"),
Collections.singletonList(seriesScanOperator1),
Collections.singletonList(Collections.singletonList(1)),
@@ -151,11 +150,11 @@ public class DeviceMergeOperatorTest {
new MeasurementPath(DEVICE_MERGE_OPERATOR_TEST_SG + ".device1.sensor1", TSDataType.INT32);
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath2,
Collections.singleton("sensor1"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
true);
@@ -166,7 +165,7 @@ public class DeviceMergeOperatorTest {
DeviceViewOperator deviceViewOperator2 =
new DeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(3),
+ driverContext.getOperatorContexts().get(3),
Collections.singletonList(DEVICE_MERGE_OPERATOR_TEST_SG + ".device1"),
Collections.singletonList(seriesScanOperator2),
Collections.singletonList(Collections.singletonList(2)),
@@ -180,7 +179,7 @@ public class DeviceMergeOperatorTest {
deviceOperators.add(deviceViewOperator2);
DeviceMergeOperator deviceMergeOperator =
new DeviceMergeOperator(
- fragmentInstanceContext.getOperatorContexts().get(4),
+ driverContext.getOperatorContexts().get(4),
devices,
deviceOperators,
dataTypes,
@@ -258,21 +257,19 @@ public class DeviceMergeOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId3 = new PlanNodeId("3");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
3, planNodeId3, DeviceViewOperatorTest.class.getSimpleName());
PlanNodeId planNodeId4 = new PlanNodeId("4");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
4, planNodeId4, DeviceViewOperatorTest.class.getSimpleName());
PlanNodeId planNodeId5 = new PlanNodeId("5");
- fragmentInstanceContext.addOperatorContext(
- 5, planNodeId5, DeviceMergeOperator.class.getSimpleName());
+ driverContext.addOperatorContext(5, planNodeId5, DeviceMergeOperator.class.getSimpleName());
List<TSDataType> dataTypes = new ArrayList<>();
dataTypes.add(TSDataType.TEXT);
@@ -281,11 +278,11 @@ public class DeviceMergeOperatorTest {
new MeasurementPath(DEVICE_MERGE_OPERATOR_TEST_SG + ".device0.sensor0", TSDataType.INT32);
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
true);
@@ -305,7 +302,7 @@ public class DeviceMergeOperatorTest {
DeviceViewOperator deviceViewOperator1 =
new DeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
Collections.singletonList(DEVICE_MERGE_OPERATOR_TEST_SG + ".device0"),
Collections.singletonList(seriesScanOperator1),
Collections.singletonList(Collections.singletonList(1)),
@@ -313,11 +310,11 @@ public class DeviceMergeOperatorTest {
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath1,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
true);
@@ -334,7 +331,7 @@ public class DeviceMergeOperatorTest {
seriesScanOperator2.initQueryDataSource(new QueryDataSource(seqResources2, unSeqResources2));
DeviceViewOperator deviceViewOperator2 =
new DeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(3),
+ driverContext.getOperatorContexts().get(3),
Collections.singletonList(DEVICE_MERGE_OPERATOR_TEST_SG + ".device0"),
Collections.singletonList(seriesScanOperator2),
Collections.singletonList(Collections.singletonList(1)),
@@ -347,7 +344,7 @@ public class DeviceMergeOperatorTest {
deviceOperators.add(deviceViewOperator2);
DeviceMergeOperator deviceMergeOperator =
new DeviceMergeOperator(
- fragmentInstanceContext.getOperatorContexts().get(4),
+ driverContext.getOperatorContexts().get(4),
devices,
deviceOperators,
dataTypes,
@@ -408,20 +405,18 @@ public class DeviceMergeOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId3 = new PlanNodeId("3");
- fragmentInstanceContext.addOperatorContext(
- 3, planNodeId3, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(3, planNodeId3, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(
4, new PlanNodeId("4"), DeviceViewOperatorTest.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
5, new PlanNodeId("5"), DeviceViewOperatorTest.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
6, new PlanNodeId("6"), DeviceMergeOperator.class.getSimpleName());
List<TSDataType> dataTypes = new ArrayList<>();
@@ -432,11 +427,11 @@ public class DeviceMergeOperatorTest {
new MeasurementPath(DEVICE_MERGE_OPERATOR_TEST_SG + ".device0.sensor0", TSDataType.INT32);
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
true);
@@ -458,11 +453,11 @@ public class DeviceMergeOperatorTest {
new MeasurementPath(DEVICE_MERGE_OPERATOR_TEST_SG + ".device1.sensor1", TSDataType.INT32);
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath2,
Collections.singleton("sensor1"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
true);
@@ -482,7 +477,7 @@ public class DeviceMergeOperatorTest {
deviceColumnIndex.add(Collections.singletonList(2));
DeviceViewOperator deviceViewOperator1 =
new DeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(3),
+ driverContext.getOperatorContexts().get(3),
devices,
deviceOperators,
deviceColumnIndex,
@@ -490,11 +485,11 @@ public class DeviceMergeOperatorTest {
SeriesScanOperator seriesScanOperator3 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(2),
planNodeId3,
measurementPath1,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(2),
null,
null,
true);
@@ -511,7 +506,7 @@ public class DeviceMergeOperatorTest {
seriesScanOperator3.initQueryDataSource(new QueryDataSource(seqResources2, unSeqResources2));
DeviceViewOperator deviceViewOperator2 =
new DeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(4),
+ driverContext.getOperatorContexts().get(4),
Collections.singletonList(DEVICE_MERGE_OPERATOR_TEST_SG + ".device0"),
Collections.singletonList(seriesScanOperator3),
Collections.singletonList(Collections.singletonList(1)),
@@ -522,7 +517,7 @@ public class DeviceMergeOperatorTest {
deviceViewOperators.add(deviceViewOperator2);
DeviceMergeOperator deviceMergeOperator =
new DeviceMergeOperator(
- fragmentInstanceContext.getOperatorContexts().get(5),
+ driverContext.getOperatorContexts().get(5),
devices,
deviceViewOperators,
dataTypes,
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/DeviceViewOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/DeviceViewOperatorTest.java
index 6073809c37..3c9aa5fd05 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/DeviceViewOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/DeviceViewOperatorTest.java
@@ -27,6 +27,7 @@ import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.DeviceViewOperator;
@@ -93,24 +94,23 @@ public class DeviceViewOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(
3, new PlanNodeId("3"), DeviceViewOperatorTest.class.getSimpleName());
MeasurementPath measurementPath1 =
new MeasurementPath(DEVICE_MERGE_OPERATOR_TEST_SG + ".device0.sensor0", TSDataType.INT32);
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
true);
@@ -123,11 +123,11 @@ public class DeviceViewOperatorTest {
new MeasurementPath(DEVICE_MERGE_OPERATOR_TEST_SG + ".device1.sensor1", TSDataType.INT32);
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath2,
Collections.singleton("sensor1"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
true);
@@ -152,7 +152,7 @@ public class DeviceViewOperatorTest {
DeviceViewOperator deviceViewOperator =
new DeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
devices,
deviceOperators,
deviceColumnIndex,
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/FillOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/FillOperatorTest.java
index 4f17cdb66d..af84ea03a8 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/FillOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/FillOperatorTest.java
@@ -22,6 +22,7 @@ import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.FillOperator;
@@ -56,9 +57,9 @@ public class FillOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, FillOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, FillOperator.class.getSimpleName());
IFill[] fillArray =
new IFill[] {
@@ -68,14 +69,14 @@ public class FillOperatorTest {
};
FillOperator fillOperator =
new FillOperator(
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
fillArray,
new Operator() {
private int index = 0;
@Override
public OperatorContext getOperatorContext() {
- return fragmentInstanceContext.getOperatorContexts().get(0);
+ return driverContext.getOperatorContexts().get(0);
}
@Override
@@ -230,22 +231,22 @@ public class FillOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, FillOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, FillOperator.class.getSimpleName());
IFill[] fillArray =
new IFill[] {new IntPreviousFill(), new IntPreviousFill(), new IntPreviousFill()};
FillOperator fillOperator =
new FillOperator(
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
fillArray,
new Operator() {
private int index = 0;
@Override
public OperatorContext getOperatorContext() {
- return fragmentInstanceContext.getOperatorContexts().get(0);
+ return driverContext.getOperatorContexts().get(0);
}
@Override
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LastQueryMergeOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LastQueryMergeOperatorTest.java
index b23c8d10fa..c409373a4d 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LastQueryMergeOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LastQueryMergeOperatorTest.java
@@ -23,6 +23,7 @@ import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.last.LastQueryMergeOperator;
@@ -71,11 +72,11 @@ public class LastQueryMergeOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, LastQueryMergeOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, LastQueryMergeOperator.class.getSimpleName());
- fragmentInstanceContext
+ driverContext
.getOperatorContexts()
.forEach(operatorContext -> operatorContext.setMaxRunTime(TEST_TIME_SLICE));
@@ -99,7 +100,7 @@ public class LastQueryMergeOperatorTest {
@Override
public OperatorContext getOperatorContext() {
- return fragmentInstanceContext.getOperatorContexts().get(0);
+ return driverContext.getOperatorContexts().get(0);
}
@Override
@@ -163,7 +164,7 @@ public class LastQueryMergeOperatorTest {
@Override
public OperatorContext getOperatorContext() {
- return fragmentInstanceContext.getOperatorContexts().get(0);
+ return driverContext.getOperatorContexts().get(0);
}
@Override
@@ -209,7 +210,7 @@ public class LastQueryMergeOperatorTest {
LastQueryMergeOperator lastQueryMergeOperator =
new LastQueryMergeOperator(
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
ImmutableList.of(operator1, operator2),
Comparator.reverseOrder());
@@ -267,11 +268,11 @@ public class LastQueryMergeOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, LastQueryMergeOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, LastQueryMergeOperator.class.getSimpleName());
- fragmentInstanceContext
+ driverContext
.getOperatorContexts()
.forEach(operatorContext -> operatorContext.setMaxRunTime(TEST_TIME_SLICE));
@@ -295,7 +296,7 @@ public class LastQueryMergeOperatorTest {
@Override
public OperatorContext getOperatorContext() {
- return fragmentInstanceContext.getOperatorContexts().get(0);
+ return driverContext.getOperatorContexts().get(0);
}
@Override
@@ -359,7 +360,7 @@ public class LastQueryMergeOperatorTest {
@Override
public OperatorContext getOperatorContext() {
- return fragmentInstanceContext.getOperatorContexts().get(0);
+ return driverContext.getOperatorContexts().get(0);
}
@Override
@@ -405,7 +406,7 @@ public class LastQueryMergeOperatorTest {
LastQueryMergeOperator lastQueryMergeOperator =
new LastQueryMergeOperator(
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
ImmutableList.of(operator1, operator2),
Comparator.naturalOrder());
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LastQueryOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LastQueryOperatorTest.java
index 7d8d9f4257..fec0651cc1 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LastQueryOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LastQueryOperatorTest.java
@@ -28,6 +28,7 @@ import org.apache.iotdb.db.mpp.aggregation.Aggregator;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.last.LastQueryOperator;
@@ -103,25 +104,25 @@ public class LastQueryOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId1, SeriesAggregationScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
2, planNodeId2, UpdateLastCacheOperator.class.getSimpleName());
PlanNodeId planNodeId3 = new PlanNodeId("3");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
3, planNodeId3, SeriesAggregationScanOperator.class.getSimpleName());
PlanNodeId planNodeId4 = new PlanNodeId("4");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
4, planNodeId4, UpdateLastCacheOperator.class.getSimpleName());
PlanNodeId planNodeId5 = new PlanNodeId("5");
- fragmentInstanceContext.addOperatorContext(
- 5, planNodeId5, LastQueryOperator.class.getSimpleName());
+ driverContext.addOperatorContext(5, planNodeId5, LastQueryOperator.class.getSimpleName());
- fragmentInstanceContext
+ driverContext
.getOperatorContexts()
.forEach(
operatorContext -> {
@@ -133,7 +134,7 @@ public class LastQueryOperatorTest {
planNodeId1,
measurementPath1,
allSensors,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
aggregators1,
initTimeRangeIterator(null, false, true),
null,
@@ -145,7 +146,7 @@ public class LastQueryOperatorTest {
UpdateLastCacheOperator updateLastCacheOperator1 =
new UpdateLastCacheOperator(
- fragmentInstanceContext.getOperatorContexts().get(1),
+ driverContext.getOperatorContexts().get(1),
seriesAggregationScanOperator1,
measurementPath1,
measurementPath1.getSeriesType(),
@@ -157,7 +158,7 @@ public class LastQueryOperatorTest {
planNodeId3,
measurementPath2,
allSensors,
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
aggregators2,
initTimeRangeIterator(null, false, true),
null,
@@ -169,7 +170,7 @@ public class LastQueryOperatorTest {
UpdateLastCacheOperator updateLastCacheOperator2 =
new UpdateLastCacheOperator(
- fragmentInstanceContext.getOperatorContexts().get(3),
+ driverContext.getOperatorContexts().get(3),
seriesAggregationScanOperator2,
measurementPath2,
measurementPath2.getSeriesType(),
@@ -178,7 +179,7 @@ public class LastQueryOperatorTest {
LastQueryOperator lastQueryOperator =
new LastQueryOperator(
- fragmentInstanceContext.getOperatorContexts().get(4),
+ driverContext.getOperatorContexts().get(4),
ImmutableList.of(updateLastCacheOperator1, updateLastCacheOperator2),
LastQueryUtil.createTsBlockBuilder());
@@ -226,25 +227,25 @@ public class LastQueryOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId1, SeriesAggregationScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
2, planNodeId2, UpdateLastCacheOperator.class.getSimpleName());
PlanNodeId planNodeId3 = new PlanNodeId("3");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
3, planNodeId3, SeriesAggregationScanOperator.class.getSimpleName());
PlanNodeId planNodeId4 = new PlanNodeId("4");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
4, planNodeId4, UpdateLastCacheOperator.class.getSimpleName());
PlanNodeId planNodeId5 = new PlanNodeId("5");
- fragmentInstanceContext.addOperatorContext(
- 5, planNodeId4, LastQueryOperator.class.getSimpleName());
+ driverContext.addOperatorContext(5, planNodeId4, LastQueryOperator.class.getSimpleName());
- fragmentInstanceContext
+ driverContext
.getOperatorContexts()
.forEach(
operatorContext -> {
@@ -256,7 +257,7 @@ public class LastQueryOperatorTest {
planNodeId1,
measurementPath1,
allSensors,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
aggregators1,
initTimeRangeIterator(null, false, true),
null,
@@ -268,7 +269,7 @@ public class LastQueryOperatorTest {
UpdateLastCacheOperator updateLastCacheOperator1 =
new UpdateLastCacheOperator(
- fragmentInstanceContext.getOperatorContexts().get(1),
+ driverContext.getOperatorContexts().get(1),
seriesAggregationScanOperator1,
measurementPath1,
measurementPath1.getSeriesType(),
@@ -280,7 +281,7 @@ public class LastQueryOperatorTest {
planNodeId3,
measurementPath2,
allSensors,
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
aggregators2,
initTimeRangeIterator(null, false, true),
null,
@@ -292,7 +293,7 @@ public class LastQueryOperatorTest {
UpdateLastCacheOperator updateLastCacheOperator2 =
new UpdateLastCacheOperator(
- fragmentInstanceContext.getOperatorContexts().get(3),
+ driverContext.getOperatorContexts().get(3),
seriesAggregationScanOperator2,
measurementPath2,
measurementPath2.getSeriesType(),
@@ -310,7 +311,7 @@ public class LastQueryOperatorTest {
LastQueryOperator lastQueryOperator =
new LastQueryOperator(
- fragmentInstanceContext.getOperatorContexts().get(4),
+ driverContext.getOperatorContexts().get(4),
ImmutableList.of(updateLastCacheOperator1, updateLastCacheOperator2),
builder);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LastQuerySortOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LastQuerySortOperatorTest.java
index e2caf939a5..26d454f8f4 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LastQuerySortOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LastQuerySortOperatorTest.java
@@ -28,6 +28,7 @@ import org.apache.iotdb.db.mpp.aggregation.Aggregator;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.last.LastQueryOperator;
@@ -105,25 +106,25 @@ public class LastQuerySortOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId1, SeriesAggregationScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
2, planNodeId2, UpdateLastCacheOperator.class.getSimpleName());
PlanNodeId planNodeId3 = new PlanNodeId("3");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
3, planNodeId3, SeriesAggregationScanOperator.class.getSimpleName());
PlanNodeId planNodeId4 = new PlanNodeId("4");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
4, planNodeId4, UpdateLastCacheOperator.class.getSimpleName());
PlanNodeId planNodeId5 = new PlanNodeId("5");
- fragmentInstanceContext.addOperatorContext(
- 5, planNodeId5, LastQueryOperator.class.getSimpleName());
+ driverContext.addOperatorContext(5, planNodeId5, LastQueryOperator.class.getSimpleName());
- fragmentInstanceContext
+ driverContext
.getOperatorContexts()
.forEach(
operatorContext -> {
@@ -135,7 +136,7 @@ public class LastQuerySortOperatorTest {
planNodeId1,
measurementPath1,
allSensors,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
aggregators1,
initTimeRangeIterator(null, false, true),
null,
@@ -147,7 +148,7 @@ public class LastQuerySortOperatorTest {
UpdateLastCacheOperator updateLastCacheOperator1 =
new UpdateLastCacheOperator(
- fragmentInstanceContext.getOperatorContexts().get(1),
+ driverContext.getOperatorContexts().get(1),
seriesAggregationScanOperator1,
measurementPath1,
measurementPath1.getSeriesType(),
@@ -159,7 +160,7 @@ public class LastQuerySortOperatorTest {
planNodeId3,
measurementPath2,
allSensors,
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
aggregators2,
initTimeRangeIterator(null, false, true),
null,
@@ -171,7 +172,7 @@ public class LastQuerySortOperatorTest {
UpdateLastCacheOperator updateLastCacheOperator2 =
new UpdateLastCacheOperator(
- fragmentInstanceContext.getOperatorContexts().get(3),
+ driverContext.getOperatorContexts().get(3),
seriesAggregationScanOperator2,
measurementPath2,
measurementPath2.getSeriesType(),
@@ -180,7 +181,7 @@ public class LastQuerySortOperatorTest {
LastQuerySortOperator lastQuerySortOperator =
new LastQuerySortOperator(
- fragmentInstanceContext.getOperatorContexts().get(4),
+ driverContext.getOperatorContexts().get(4),
LastQueryUtil.createTsBlockBuilder().build(),
ImmutableList.of(updateLastCacheOperator1, updateLastCacheOperator2),
Comparator.naturalOrder());
@@ -229,25 +230,25 @@ public class LastQuerySortOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId1, SeriesAggregationScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
2, planNodeId2, UpdateLastCacheOperator.class.getSimpleName());
PlanNodeId planNodeId3 = new PlanNodeId("3");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
3, planNodeId3, SeriesAggregationScanOperator.class.getSimpleName());
PlanNodeId planNodeId4 = new PlanNodeId("4");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
4, planNodeId4, UpdateLastCacheOperator.class.getSimpleName());
PlanNodeId planNodeId5 = new PlanNodeId("5");
- fragmentInstanceContext.addOperatorContext(
- 5, planNodeId4, LastQueryOperator.class.getSimpleName());
+ driverContext.addOperatorContext(5, planNodeId4, LastQueryOperator.class.getSimpleName());
- fragmentInstanceContext
+ driverContext
.getOperatorContexts()
.forEach(
operatorContext -> {
@@ -259,7 +260,7 @@ public class LastQuerySortOperatorTest {
planNodeId1,
measurementPath1,
allSensors,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
aggregators1,
initTimeRangeIterator(null, false, true),
null,
@@ -271,7 +272,7 @@ public class LastQuerySortOperatorTest {
UpdateLastCacheOperator updateLastCacheOperator1 =
new UpdateLastCacheOperator(
- fragmentInstanceContext.getOperatorContexts().get(1),
+ driverContext.getOperatorContexts().get(1),
seriesAggregationScanOperator1,
measurementPath1,
measurementPath1.getSeriesType(),
@@ -283,7 +284,7 @@ public class LastQuerySortOperatorTest {
planNodeId3,
measurementPath2,
allSensors,
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
aggregators2,
initTimeRangeIterator(null, false, true),
null,
@@ -295,7 +296,7 @@ public class LastQuerySortOperatorTest {
UpdateLastCacheOperator updateLastCacheOperator2 =
new UpdateLastCacheOperator(
- fragmentInstanceContext.getOperatorContexts().get(3),
+ driverContext.getOperatorContexts().get(3),
seriesAggregationScanOperator2,
measurementPath2,
measurementPath2.getSeriesType(),
@@ -313,7 +314,7 @@ public class LastQuerySortOperatorTest {
LastQuerySortOperator lastQuerySortOperator =
new LastQuerySortOperator(
- fragmentInstanceContext.getOperatorContexts().get(4),
+ driverContext.getOperatorContexts().get(4),
builder.build(),
ImmutableList.of(updateLastCacheOperator2, updateLastCacheOperator1),
Comparator.reverseOrder());
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LimitOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LimitOperatorTest.java
index 2c18a17099..9786a4472d 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LimitOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LimitOperatorTest.java
@@ -27,6 +27,7 @@ import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.LimitOperator;
@@ -100,23 +101,21 @@ public class LimitOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(
3, new PlanNodeId("3"), TimeJoinOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
- 4, new PlanNodeId("4"), LimitOperator.class.getSimpleName());
+ driverContext.addOperatorContext(4, new PlanNodeId("4"), LimitOperator.class.getSimpleName());
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
true);
@@ -129,11 +128,11 @@ public class LimitOperatorTest {
new MeasurementPath(TIME_JOIN_OPERATOR_TEST_SG + ".device0.sensor1", TSDataType.INT32);
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath2,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
true);
@@ -144,7 +143,7 @@ public class LimitOperatorTest {
TimeJoinOperator timeJoinOperator =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
Arrays.asList(seriesScanOperator1, seriesScanOperator2),
Ordering.ASC,
Arrays.asList(TSDataType.INT32, TSDataType.INT32),
@@ -154,8 +153,7 @@ public class LimitOperatorTest {
new AscTimeComparator());
LimitOperator limitOperator =
- new LimitOperator(
- fragmentInstanceContext.getOperatorContexts().get(3), 250, timeJoinOperator);
+ new LimitOperator(driverContext.getOperatorContexts().get(3), 250, timeJoinOperator);
int count = 0;
while (limitOperator.hasNext()) {
TsBlock tsBlock = limitOperator.next();
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LinearFillOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LinearFillOperatorTest.java
index 6458bffaab..6bac123f24 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LinearFillOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/LinearFillOperatorTest.java
@@ -22,6 +22,7 @@ import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.LinearFillOperator;
@@ -58,9 +59,9 @@ public class LinearFillOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, LinearFillOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, LinearFillOperator.class.getSimpleName());
LinearFill[] fillArray =
new LinearFill[] {
@@ -71,7 +72,7 @@ public class LinearFillOperatorTest {
};
LinearFillOperator fillOperator =
new LinearFillOperator(
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
fillArray,
new Operator() {
private int index = 0;
@@ -126,7 +127,7 @@ public class LinearFillOperatorTest {
@Override
public OperatorContext getOperatorContext() {
- return fragmentInstanceContext.getOperatorContexts().get(0);
+ return driverContext.getOperatorContexts().get(0);
}
@Override
@@ -271,9 +272,9 @@ public class LinearFillOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, LinearFillOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, LinearFillOperator.class.getSimpleName());
LinearFill[] fillArray =
new LinearFill[] {
@@ -284,7 +285,7 @@ public class LinearFillOperatorTest {
};
LinearFillOperator fillOperator =
new LinearFillOperator(
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
fillArray,
new Operator() {
private int index = 0;
@@ -339,7 +340,7 @@ public class LinearFillOperatorTest {
@Override
public OperatorContext getOperatorContext() {
- return fragmentInstanceContext.getOperatorContexts().get(0);
+ return driverContext.getOperatorContexts().get(0);
}
@Override
@@ -484,9 +485,9 @@ public class LinearFillOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, LinearFillOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, LinearFillOperator.class.getSimpleName());
LinearFill[] fillArray =
new LinearFill[] {
@@ -497,7 +498,7 @@ public class LinearFillOperatorTest {
};
LinearFillOperator fillOperator =
new LinearFillOperator(
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
fillArray,
new Operator() {
private int index = 0;
@@ -552,7 +553,7 @@ public class LinearFillOperatorTest {
@Override
public OperatorContext getOperatorContext() {
- return fragmentInstanceContext.getOperatorContexts().get(0);
+ return driverContext.getOperatorContexts().get(0);
}
@Override
@@ -697,9 +698,9 @@ public class LinearFillOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, LinearFillOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, LinearFillOperator.class.getSimpleName());
LinearFill[] fillArray =
new LinearFill[] {
@@ -710,7 +711,7 @@ public class LinearFillOperatorTest {
};
LinearFillOperator fillOperator =
new LinearFillOperator(
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
fillArray,
new Operator() {
private int index = 0;
@@ -765,7 +766,7 @@ public class LinearFillOperatorTest {
@Override
public OperatorContext getOperatorContext() {
- return fragmentInstanceContext.getOperatorContexts().get(0);
+ return driverContext.getOperatorContexts().get(0);
}
@Override
@@ -910,14 +911,14 @@ public class LinearFillOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, LinearFillOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, LinearFillOperator.class.getSimpleName());
LinearFill[] fillArray = new LinearFill[] {new FloatLinearFill()};
LinearFillOperator fillOperator =
new LinearFillOperator(
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
fillArray,
new Operator() {
private int index = 0;
@@ -932,7 +933,7 @@ public class LinearFillOperatorTest {
@Override
public OperatorContext getOperatorContext() {
- return fragmentInstanceContext.getOperatorContexts().get(0);
+ return driverContext.getOperatorContexts().get(0);
}
@Override
@@ -1030,14 +1031,14 @@ public class LinearFillOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, LinearFillOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, LinearFillOperator.class.getSimpleName());
LinearFill[] fillArray = new LinearFill[] {new FloatLinearFill()};
LinearFillOperator fillOperator =
new LinearFillOperator(
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
fillArray,
new Operator() {
private int index = 0;
@@ -1052,7 +1053,7 @@ public class LinearFillOperatorTest {
@Override
public OperatorContext getOperatorContext() {
- return fragmentInstanceContext.getOperatorContexts().get(0);
+ return driverContext.getOperatorContexts().get(0);
}
@Override
@@ -1150,14 +1151,14 @@ public class LinearFillOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, LinearFillOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, LinearFillOperator.class.getSimpleName());
ILinearFill[] fillArray = new ILinearFill[] {new IdentityLinearFill()};
LinearFillOperator fillOperator =
new LinearFillOperator(
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
fillArray,
new Operator() {
private int index = 0;
@@ -1172,7 +1173,7 @@ public class LinearFillOperatorTest {
@Override
public OperatorContext getOperatorContext() {
- return fragmentInstanceContext.getOperatorContexts().get(0);
+ return driverContext.getOperatorContexts().get(0);
}
@Override
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/MergeSortOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/MergeSortOperatorTest.java
index 58dd3932aa..39ff8050be 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/MergeSortOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/MergeSortOperatorTest.java
@@ -31,6 +31,7 @@ import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
import org.apache.iotdb.db.mpp.common.header.DatasetHeader;
import org.apache.iotdb.db.mpp.common.header.DatasetHeaderFactory;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.DeviceViewOperator;
@@ -152,32 +153,28 @@ public class MergeSortOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId3 = new PlanNodeId("3");
- fragmentInstanceContext.addOperatorContext(
- 3, planNodeId3, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(3, planNodeId3, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId4 = new PlanNodeId("4");
- fragmentInstanceContext.addOperatorContext(
- 4, planNodeId4, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(4, planNodeId4, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId5 = new PlanNodeId("5");
- fragmentInstanceContext.addOperatorContext(
- 5, planNodeId5, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(5, planNodeId5, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(
6, new PlanNodeId("6"), SingleDeviceViewOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
7, new PlanNodeId("7"), TimeJoinOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
8, new PlanNodeId("8"), SingleDeviceViewOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
9, new PlanNodeId("9"), TimeJoinOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
10, new PlanNodeId("10"), SingleDeviceViewOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
11, new PlanNodeId("11"), MergeSortOperator.class.getSimpleName());
MeasurementPath measurementPath1 =
@@ -192,11 +189,11 @@ public class MergeSortOperatorTest {
new MeasurementPath(MERGE_SORT_OPERATOR_TEST_SG + ".device2.sensor1", TSDataType.INT32);
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
timeOrdering == Ordering.ASC);
@@ -206,11 +203,11 @@ public class MergeSortOperatorTest {
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath2,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
timeOrdering == Ordering.ASC);
@@ -220,11 +217,11 @@ public class MergeSortOperatorTest {
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
SeriesScanOperator seriesScanOperator3 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(2),
planNodeId3,
measurementPath3,
Collections.singleton("sensor1"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(2),
null,
null,
timeOrdering == Ordering.ASC);
@@ -234,11 +231,11 @@ public class MergeSortOperatorTest {
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
SeriesScanOperator seriesScanOperator4 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(3),
planNodeId4,
measurementPath4,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(3),
null,
null,
timeOrdering == Ordering.ASC);
@@ -248,11 +245,11 @@ public class MergeSortOperatorTest {
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
SeriesScanOperator seriesScanOperator5 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(4),
planNodeId5,
measurementPath5,
Collections.singleton("sensor1"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(4),
null,
null,
timeOrdering == Ordering.ASC);
@@ -272,7 +269,7 @@ public class MergeSortOperatorTest {
TSDataType.INT32));
SingleDeviceViewOperator singleDeviceViewOperator1 =
new SingleDeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(5),
+ driverContext.getOperatorContexts().get(5),
DEVICE0,
seriesScanOperator1,
Collections.singletonList(1),
@@ -280,7 +277,7 @@ public class MergeSortOperatorTest {
TimeJoinOperator timeJoinOperator1 =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(6),
+ driverContext.getOperatorContexts().get(6),
Arrays.asList(seriesScanOperator2, seriesScanOperator3),
timeOrdering,
Arrays.asList(TSDataType.INT32, TSDataType.INT32),
@@ -298,7 +295,7 @@ public class MergeSortOperatorTest {
timeOrdering == Ordering.ASC ? new AscTimeComparator() : new DescTimeComparator());
SingleDeviceViewOperator singleDeviceViewOperator2 =
new SingleDeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(7),
+ driverContext.getOperatorContexts().get(7),
DEVICE1,
timeJoinOperator1,
Arrays.asList(2, 3),
@@ -306,7 +303,7 @@ public class MergeSortOperatorTest {
TimeJoinOperator timeJoinOperator2 =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(8),
+ driverContext.getOperatorContexts().get(8),
Arrays.asList(seriesScanOperator4, seriesScanOperator5),
timeOrdering,
Arrays.asList(TSDataType.INT32, TSDataType.INT32),
@@ -324,14 +321,14 @@ public class MergeSortOperatorTest {
timeOrdering == Ordering.ASC ? new AscTimeComparator() : new DescTimeComparator());
SingleDeviceViewOperator singleDeviceViewOperator3 =
new SingleDeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(9),
+ driverContext.getOperatorContexts().get(9),
DEVICE2,
timeJoinOperator2,
Arrays.asList(4, 5),
tsDataTypes);
return new MergeSortOperator(
- fragmentInstanceContext.getOperatorContexts().get(10),
+ driverContext.getOperatorContexts().get(10),
Arrays.asList(
singleDeviceViewOperator1, singleDeviceViewOperator2, singleDeviceViewOperator3),
tsDataTypes,
@@ -561,47 +558,41 @@ public class MergeSortOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId3 = new PlanNodeId("3");
- fragmentInstanceContext.addOperatorContext(
- 3, planNodeId3, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(3, planNodeId3, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId4 = new PlanNodeId("4");
- fragmentInstanceContext.addOperatorContext(
- 4, planNodeId4, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(4, planNodeId4, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId5 = new PlanNodeId("5");
- fragmentInstanceContext.addOperatorContext(
- 5, planNodeId5, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(5, planNodeId5, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId6 = new PlanNodeId("6");
- fragmentInstanceContext.addOperatorContext(
- 6, planNodeId6, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(6, planNodeId6, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId7 = new PlanNodeId("7");
- fragmentInstanceContext.addOperatorContext(
- 7, planNodeId7, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(7, planNodeId7, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
8, new PlanNodeId("8"), TimeJoinOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
9, new PlanNodeId("9"), TimeJoinOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
10, new PlanNodeId("10"), TimeJoinOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
11, new PlanNodeId("11"), SingleDeviceViewOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
12, new PlanNodeId("12"), SingleDeviceViewOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
13, new PlanNodeId("13"), SingleDeviceViewOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
14, new PlanNodeId("14"), SingleDeviceViewOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
15, new PlanNodeId("15"), MergeSortOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
16, new PlanNodeId("16"), MergeSortOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
17, new PlanNodeId("17"), MergeSortOperator.class.getSimpleName());
MeasurementPath measurementPath1 =
@@ -621,11 +612,11 @@ public class MergeSortOperatorTest {
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
timeOrdering == Ordering.ASC);
@@ -635,11 +626,11 @@ public class MergeSortOperatorTest {
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath2,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
timeOrdering == Ordering.ASC);
@@ -649,11 +640,11 @@ public class MergeSortOperatorTest {
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
SeriesScanOperator seriesScanOperator3 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(2),
planNodeId3,
measurementPath3,
Collections.singleton("sensor1"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(2),
null,
null,
timeOrdering == Ordering.ASC);
@@ -663,11 +654,11 @@ public class MergeSortOperatorTest {
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
SeriesScanOperator seriesScanOperator4 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(3),
planNodeId4,
measurementPath4,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(3),
null,
null,
timeOrdering == Ordering.ASC);
@@ -677,11 +668,11 @@ public class MergeSortOperatorTest {
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
SeriesScanOperator seriesScanOperator5 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(4),
planNodeId5,
measurementPath5,
Collections.singleton("sensor1"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(4),
null,
null,
timeOrdering == Ordering.ASC);
@@ -691,11 +682,11 @@ public class MergeSortOperatorTest {
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
SeriesScanOperator seriesScanOperator6 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(5),
planNodeId6,
measurementPath6,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(5),
null,
null,
timeOrdering == Ordering.ASC);
@@ -705,11 +696,11 @@ public class MergeSortOperatorTest {
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
SeriesScanOperator seriesScanOperator7 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(6),
planNodeId7,
measurementPath7,
Collections.singleton("sensor1"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(6),
null,
null,
timeOrdering == Ordering.ASC);
@@ -723,7 +714,7 @@ public class MergeSortOperatorTest {
TimeJoinOperator timeJoinOperator1 =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(7),
+ driverContext.getOperatorContexts().get(7),
Arrays.asList(seriesScanOperator2, seriesScanOperator3),
timeOrdering,
Arrays.asList(TSDataType.INT32, TSDataType.INT32),
@@ -742,7 +733,7 @@ public class MergeSortOperatorTest {
TimeJoinOperator timeJoinOperator2 =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(8),
+ driverContext.getOperatorContexts().get(8),
Arrays.asList(seriesScanOperator4, seriesScanOperator5),
timeOrdering,
Arrays.asList(TSDataType.INT32, TSDataType.INT32),
@@ -761,7 +752,7 @@ public class MergeSortOperatorTest {
TimeJoinOperator timeJoinOperator3 =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(9),
+ driverContext.getOperatorContexts().get(9),
Arrays.asList(seriesScanOperator6, seriesScanOperator7),
timeOrdering,
Arrays.asList(TSDataType.INT32, TSDataType.INT32),
@@ -779,28 +770,28 @@ public class MergeSortOperatorTest {
timeOrdering == Ordering.ASC ? new AscTimeComparator() : new DescTimeComparator());
SingleDeviceViewOperator singleDeviceViewOperator1 =
new SingleDeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(10),
+ driverContext.getOperatorContexts().get(10),
DEVICE0,
seriesScanOperator1,
Collections.singletonList(1),
tsDataTypes);
SingleDeviceViewOperator singleDeviceViewOperator2 =
new SingleDeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(11),
+ driverContext.getOperatorContexts().get(11),
DEVICE1,
timeJoinOperator1,
Arrays.asList(1, 2),
tsDataTypes);
SingleDeviceViewOperator singleDeviceViewOperator3 =
new SingleDeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(12),
+ driverContext.getOperatorContexts().get(12),
DEVICE2,
timeJoinOperator2,
Arrays.asList(1, 2),
tsDataTypes);
SingleDeviceViewOperator singleDeviceViewOperator4 =
new SingleDeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(13),
+ driverContext.getOperatorContexts().get(13),
DEVICE3,
timeJoinOperator3,
Arrays.asList(1, 2),
@@ -808,7 +799,7 @@ public class MergeSortOperatorTest {
MergeSortOperator mergeSortOperator1 =
new MergeSortOperator(
- fragmentInstanceContext.getOperatorContexts().get(14),
+ driverContext.getOperatorContexts().get(14),
Arrays.asList(singleDeviceViewOperator1, singleDeviceViewOperator2),
tsDataTypes,
MergeSortComparator.getComparator(
@@ -819,7 +810,7 @@ public class MergeSortOperatorTest {
null));
MergeSortOperator mergeSortOperator2 =
new MergeSortOperator(
- fragmentInstanceContext.getOperatorContexts().get(15),
+ driverContext.getOperatorContexts().get(15),
Arrays.asList(singleDeviceViewOperator3, singleDeviceViewOperator4),
tsDataTypes,
MergeSortComparator.getComparator(
@@ -830,7 +821,7 @@ public class MergeSortOperatorTest {
null));
return new MergeSortOperator(
- fragmentInstanceContext.getOperatorContexts().get(16),
+ driverContext.getOperatorContexts().get(16),
Arrays.asList(mergeSortOperator1, mergeSortOperator2),
tsDataTypes,
MergeSortComparator.getComparator(
@@ -1039,39 +1030,33 @@ public class MergeSortOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId3 = new PlanNodeId("3");
- fragmentInstanceContext.addOperatorContext(
- 3, planNodeId3, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(3, planNodeId3, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId4 = new PlanNodeId("4");
- fragmentInstanceContext.addOperatorContext(
- 4, planNodeId4, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(4, planNodeId4, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId5 = new PlanNodeId("5");
- fragmentInstanceContext.addOperatorContext(
- 5, planNodeId5, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(5, planNodeId5, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId6 = new PlanNodeId("6");
- fragmentInstanceContext.addOperatorContext(
- 6, planNodeId6, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(6, planNodeId6, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId7 = new PlanNodeId("7");
- fragmentInstanceContext.addOperatorContext(
- 7, planNodeId7, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(7, planNodeId7, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
8, new PlanNodeId("8"), TimeJoinOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
9, new PlanNodeId("9"), TimeJoinOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
10, new PlanNodeId("10"), TimeJoinOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
11, new PlanNodeId("11"), DeviceViewOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
12, new PlanNodeId("12"), DeviceViewOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
13, new PlanNodeId("13"), MergeSortOperator.class.getSimpleName());
MeasurementPath measurementPath1 =
@@ -1091,11 +1076,11 @@ public class MergeSortOperatorTest {
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
timeOrdering == Ordering.ASC);
@@ -1105,11 +1090,11 @@ public class MergeSortOperatorTest {
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath2,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
timeOrdering == Ordering.ASC);
@@ -1119,11 +1104,11 @@ public class MergeSortOperatorTest {
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
SeriesScanOperator seriesScanOperator3 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(2),
planNodeId3,
measurementPath3,
Collections.singleton("sensor1"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(2),
null,
null,
timeOrdering == Ordering.ASC);
@@ -1133,11 +1118,11 @@ public class MergeSortOperatorTest {
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
SeriesScanOperator seriesScanOperator4 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(3),
planNodeId4,
measurementPath4,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(3),
null,
null,
timeOrdering == Ordering.ASC);
@@ -1147,11 +1132,11 @@ public class MergeSortOperatorTest {
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
SeriesScanOperator seriesScanOperator5 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(4),
planNodeId5,
measurementPath5,
Collections.singleton("sensor1"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(4),
null,
null,
timeOrdering == Ordering.ASC);
@@ -1161,11 +1146,11 @@ public class MergeSortOperatorTest {
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
SeriesScanOperator seriesScanOperator6 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(5),
planNodeId6,
measurementPath6,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(5),
null,
null,
timeOrdering == Ordering.ASC);
@@ -1175,11 +1160,11 @@ public class MergeSortOperatorTest {
.setMaxRunTime(new Duration(500, TimeUnit.MILLISECONDS));
SeriesScanOperator seriesScanOperator7 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(6),
planNodeId7,
measurementPath7,
Collections.singleton("sensor1"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(6),
null,
null,
timeOrdering == Ordering.ASC);
@@ -1193,7 +1178,7 @@ public class MergeSortOperatorTest {
TimeJoinOperator timeJoinOperator1 =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(7),
+ driverContext.getOperatorContexts().get(7),
Arrays.asList(seriesScanOperator2, seriesScanOperator3),
timeOrdering,
Arrays.asList(TSDataType.INT32, TSDataType.INT32),
@@ -1212,7 +1197,7 @@ public class MergeSortOperatorTest {
TimeJoinOperator timeJoinOperator2 =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(8),
+ driverContext.getOperatorContexts().get(8),
Arrays.asList(seriesScanOperator4, seriesScanOperator5),
timeOrdering,
Arrays.asList(TSDataType.INT32, TSDataType.INT32),
@@ -1231,7 +1216,7 @@ public class MergeSortOperatorTest {
TimeJoinOperator timeJoinOperator3 =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(9),
+ driverContext.getOperatorContexts().get(9),
Arrays.asList(seriesScanOperator6, seriesScanOperator7),
timeOrdering,
Arrays.asList(TSDataType.INT32, TSDataType.INT32),
@@ -1256,7 +1241,7 @@ public class MergeSortOperatorTest {
if (deviceOrdering == Ordering.DESC) Collections.reverse(deviceColumnIndex);
DeviceViewOperator deviceViewOperator1 =
new DeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(10),
+ driverContext.getOperatorContexts().get(10),
deviceOrdering == Ordering.ASC
? Arrays.asList(DEVICE0, DEVICE1)
: Arrays.asList(DEVICE1, DEVICE0),
@@ -1270,7 +1255,7 @@ public class MergeSortOperatorTest {
deviceColumnIndex.add(Arrays.asList(1, 2));
DeviceViewOperator deviceViewOperator2 =
new DeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(11),
+ driverContext.getOperatorContexts().get(11),
deviceOrdering == Ordering.ASC
? Arrays.asList(DEVICE2, DEVICE3)
: Arrays.asList(DEVICE3, DEVICE2),
@@ -1280,7 +1265,7 @@ public class MergeSortOperatorTest {
deviceColumnIndex,
tsDataTypes);
return new MergeSortOperator(
- fragmentInstanceContext.getOperatorContexts().get(12),
+ driverContext.getOperatorContexts().get(12),
Arrays.asList(deviceViewOperator1, deviceViewOperator2),
tsDataTypes,
MergeSortComparator.getComparator(
@@ -1530,23 +1515,19 @@ public class MergeSortOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId0 = new PlanNodeId("0");
- fragmentInstanceContext.addOperatorContext(
- 0, planNodeId0, ShowQueriesOperator.class.getSimpleName());
+ driverContext.addOperatorContext(0, planNodeId0, ShowQueriesOperator.class.getSimpleName());
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, ShowQueriesOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, ShowQueriesOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SortOperator.class.getSimpleName());
+ driverContext.addOperatorContext(2, planNodeId2, SortOperator.class.getSimpleName());
PlanNodeId planNodeId3 = new PlanNodeId("3");
- fragmentInstanceContext.addOperatorContext(
- 3, planNodeId3, SortOperator.class.getSimpleName());
+ driverContext.addOperatorContext(3, planNodeId3, SortOperator.class.getSimpleName());
PlanNodeId planNodeId4 = new PlanNodeId("4");
- fragmentInstanceContext.addOperatorContext(
- 4, planNodeId4, MergeSortOperator.class.getSimpleName());
+ driverContext.addOperatorContext(4, planNodeId4, MergeSortOperator.class.getSimpleName());
- List<OperatorContext> operatorContexts = fragmentInstanceContext.getOperatorContexts();
+ List<OperatorContext> operatorContexts = driverContext.getOperatorContexts();
List<TSDataType> dataTypes = DatasetHeaderFactory.getShowQueriesHeader().getRespDataTypes();
Comparator<MergeSortKey> comparator =
MergeSortComparator.getComparator(
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OffsetOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OffsetOperatorTest.java
index 77ba187f9a..3bebde7ed1 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OffsetOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OffsetOperatorTest.java
@@ -27,6 +27,7 @@ import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.LimitOperator;
@@ -101,25 +102,23 @@ public class OffsetOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(
3, new PlanNodeId("3"), TimeJoinOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
4, new PlanNodeId("4"), OffsetOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
- 5, new PlanNodeId("5"), LimitOperator.class.getSimpleName());
+ driverContext.addOperatorContext(5, new PlanNodeId("5"), LimitOperator.class.getSimpleName());
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
true);
@@ -132,11 +131,11 @@ public class OffsetOperatorTest {
new MeasurementPath(TIME_JOIN_OPERATOR_TEST_SG + ".device0.sensor1", TSDataType.INT32);
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath2,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
true);
@@ -147,7 +146,7 @@ public class OffsetOperatorTest {
TimeJoinOperator timeJoinOperator =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
Arrays.asList(seriesScanOperator1, seriesScanOperator2),
Ordering.ASC,
Arrays.asList(TSDataType.INT32, TSDataType.INT32),
@@ -157,12 +156,10 @@ public class OffsetOperatorTest {
new AscTimeComparator());
OffsetOperator offsetOperator =
- new OffsetOperator(
- fragmentInstanceContext.getOperatorContexts().get(3), 100, timeJoinOperator);
+ new OffsetOperator(driverContext.getOperatorContexts().get(3), 100, timeJoinOperator);
LimitOperator limitOperator =
- new LimitOperator(
- fragmentInstanceContext.getOperatorContexts().get(4), 250, offsetOperator);
+ new LimitOperator(driverContext.getOperatorContexts().get(4), 250, offsetOperator);
int count = 100;
while (limitOperator.hasNext()) {
TsBlock tsBlock = limitOperator.next();
@@ -213,25 +210,23 @@ public class OffsetOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(
3, new PlanNodeId("3"), TimeJoinOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
4, new PlanNodeId("4"), OffsetOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
- 5, new PlanNodeId("5"), LimitOperator.class.getSimpleName());
+ driverContext.addOperatorContext(5, new PlanNodeId("5"), LimitOperator.class.getSimpleName());
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
true);
@@ -244,11 +239,11 @@ public class OffsetOperatorTest {
new MeasurementPath(TIME_JOIN_OPERATOR_TEST_SG + ".device0.sensor1", TSDataType.INT32);
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath2,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
true);
@@ -259,7 +254,7 @@ public class OffsetOperatorTest {
TimeJoinOperator timeJoinOperator =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
Arrays.asList(seriesScanOperator1, seriesScanOperator2),
Ordering.ASC,
Arrays.asList(TSDataType.INT32, TSDataType.INT32),
@@ -269,8 +264,7 @@ public class OffsetOperatorTest {
new AscTimeComparator());
OffsetOperator offsetOperator =
- new OffsetOperator(
- fragmentInstanceContext.getOperatorContexts().get(3), 0, timeJoinOperator);
+ new OffsetOperator(driverContext.getOperatorContexts().get(3), 0, timeJoinOperator);
int count = 0;
while (offsetOperator.hasNext()) {
@@ -321,25 +315,23 @@ public class OffsetOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(
3, new PlanNodeId("3"), TimeJoinOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
4, new PlanNodeId("4"), OffsetOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
- 5, new PlanNodeId("5"), LimitOperator.class.getSimpleName());
+ driverContext.addOperatorContext(5, new PlanNodeId("5"), LimitOperator.class.getSimpleName());
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
true);
@@ -352,11 +344,11 @@ public class OffsetOperatorTest {
new MeasurementPath(TIME_JOIN_OPERATOR_TEST_SG + ".device0.sensor1", TSDataType.INT32);
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath2,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
true);
@@ -367,7 +359,7 @@ public class OffsetOperatorTest {
TimeJoinOperator timeJoinOperator =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
Arrays.asList(seriesScanOperator1, seriesScanOperator2),
Ordering.ASC,
Arrays.asList(TSDataType.INT32, TSDataType.INT32),
@@ -377,8 +369,7 @@ public class OffsetOperatorTest {
new AscTimeComparator());
OffsetOperator offsetOperator =
- new OffsetOperator(
- fragmentInstanceContext.getOperatorContexts().get(3), 500, timeJoinOperator);
+ new OffsetOperator(driverContext.getOperatorContexts().get(3), 500, timeJoinOperator);
while (offsetOperator.hasNext()) {
TsBlock tsBlock = offsetOperator.next();
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java
index 55a7935340..ac812e0b17 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java
@@ -28,6 +28,7 @@ import org.apache.iotdb.db.mpp.aggregation.timerangeiterator.ITimeRangeIterator;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.AggregationOperator;
@@ -131,17 +132,17 @@ public class OperatorMemoryTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId, SeriesScanOperator.class.getSimpleName());
SeriesScanOperator seriesScanOperator =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId,
measurementPath,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
true);
@@ -178,15 +179,16 @@ public class OperatorMemoryTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId, AlignedSeriesScanOperator.class.getSimpleName());
AlignedSeriesScanOperator seriesScanOperator =
new AlignedSeriesScanOperator(
planNodeId,
alignedPath,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
null,
null,
true);
@@ -686,14 +688,15 @@ public class OperatorMemoryTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId, SchemaQueryScanOperator.class.getSimpleName());
SchemaQueryScanOperator<?> operator =
new SchemaQueryScanOperator<>(
planNodeId,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
Mockito.mock(ISchemaSource.class));
assertEquals(DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES, operator.calculateMaxPeekMemory());
@@ -717,14 +720,14 @@ public class OperatorMemoryTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId, SeriesScanOperator.class.getSimpleName());
CountGroupByLevelScanOperator<?> operator =
new CountGroupByLevelScanOperator<>(
planNodeId,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
4,
Mockito.mock(ISchemaSource.class));
@@ -749,14 +752,14 @@ public class OperatorMemoryTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId, SchemaCountOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId, SchemaCountOperator.class.getSimpleName());
SchemaCountOperator<?> operator =
new SchemaCountOperator<>(
planNodeId,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
Mockito.mock(ISchemaSource.class));
assertEquals(8L, operator.calculateMaxPeekMemory());
@@ -838,18 +841,13 @@ public class OperatorMemoryTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId, SeriesScanOperator.class.getSimpleName());
SchemaFetchScanOperator operator =
new SchemaFetchScanOperator(
- planNodeId,
- fragmentInstanceContext.getOperatorContexts().get(0),
- null,
- null,
- null,
- false);
+ planNodeId, driverContext.getOperatorContexts().get(0), null, null, null, false);
assertEquals(DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES, operator.calculateMaxPeekMemory());
assertEquals(DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES, operator.calculateMaxReturnSize());
@@ -1200,9 +1198,9 @@ public class OperatorMemoryTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId, SeriesScanOperator.class.getSimpleName());
List<Aggregator> aggregators = new ArrayList<>();
aggregationDescriptors.forEach(
@@ -1222,7 +1220,7 @@ public class OperatorMemoryTest {
planNodeId,
measurementPath,
allSensors,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
aggregators,
timeRangeIterator,
null,
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/RawDataAggregationOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/RawDataAggregationOperatorTest.java
index 3540d0901a..e4e70431e1 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/RawDataAggregationOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/RawDataAggregationOperatorTest.java
@@ -31,6 +31,7 @@ import org.apache.iotdb.db.mpp.aggregation.Aggregator;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.RawDataAggregationOperator;
@@ -322,17 +323,16 @@ public class RawDataAggregationOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(
3, new PlanNodeId("3"), TimeJoinOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
4, new PlanNodeId("4"), RawDataAggregationOperatorTest.class.getSimpleName());
- fragmentInstanceContext
+ driverContext
.getOperatorContexts()
.forEach(
operatorContext -> {
@@ -341,11 +341,11 @@ public class RawDataAggregationOperatorTest {
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
true);
@@ -355,11 +355,11 @@ public class RawDataAggregationOperatorTest {
new MeasurementPath(AGGREGATION_OPERATOR_TEST_SG + ".device0.sensor1", TSDataType.INT32);
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath2,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
true);
@@ -367,7 +367,7 @@ public class RawDataAggregationOperatorTest {
TimeJoinOperator timeJoinOperator =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
Arrays.asList(seriesScanOperator1, seriesScanOperator2),
Ordering.ASC,
Arrays.asList(TSDataType.INT32, TSDataType.INT32),
@@ -384,7 +384,7 @@ public class RawDataAggregationOperatorTest {
new Aggregator(accumulators.get(i), AggregationStep.SINGLE, inputLocations.get(i)));
}
return new RawDataAggregationOperator(
- fragmentInstanceContext.getOperatorContexts().get(3),
+ driverContext.getOperatorContexts().get(3),
aggregators,
initTimeRangeIterator(groupByTimeParameter, true, true),
timeJoinOperator,
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/SeriesAggregationScanOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/SeriesAggregationScanOperatorTest.java
index 15c85499b4..dbaf292251 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/SeriesAggregationScanOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/SeriesAggregationScanOperatorTest.java
@@ -29,6 +29,7 @@ import org.apache.iotdb.db.mpp.aggregation.Aggregator;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.source.SeriesAggregationScanOperator;
@@ -500,10 +501,11 @@ public class SeriesAggregationScanOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId, SeriesAggregationScanOperator.class.getSimpleName());
- fragmentInstanceContext
+ driverContext
.getOperatorContexts()
.forEach(
operatorContext -> {
@@ -515,7 +517,7 @@ public class SeriesAggregationScanOperatorTest {
planNodeId,
measurementPath,
allSensors,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
aggregators,
initTimeRangeIterator(groupByTimeParameter, ascending, true),
timeFilter,
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/SeriesScanOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/SeriesScanOperatorTest.java
index 95b2f0a788..c93b49c238 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/SeriesScanOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/SeriesScanOperatorTest.java
@@ -27,6 +27,7 @@ import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.source.SeriesScanOperator;
@@ -57,6 +58,7 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class SeriesScanOperatorTest {
+
private static final String SERIES_SCAN_OPERATOR_TEST_SG = "root.SeriesScanOperatorTest";
private final List<String> deviceIds = new ArrayList<>();
private final List<MeasurementSchema> measurementSchemas = new ArrayList<>();
@@ -90,17 +92,17 @@ public class SeriesScanOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId, SeriesScanOperator.class.getSimpleName());
SeriesScanOperator seriesScanOperator =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId,
measurementPath,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
true);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/SingleDeviceViewOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/SingleDeviceViewOperatorTest.java
index b77aed1e0b..d9b2414335 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/SingleDeviceViewOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/SingleDeviceViewOperatorTest.java
@@ -27,6 +27,7 @@ import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.SingleDeviceViewOperator;
@@ -103,16 +104,15 @@ public class SingleDeviceViewOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(
3, new PlanNodeId("3"), TimeJoinOperatorTest.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
4, new PlanNodeId("4"), SingleDeviceViewOperator.class.getSimpleName());
MeasurementPath measurementPath1 =
@@ -123,11 +123,11 @@ public class SingleDeviceViewOperatorTest {
SINGLE_DEVICE_MERGE_OPERATOR_TEST_SG + ".device0.sensor1", TSDataType.INT32);
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
Collections.singleton("sensor0"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
true);
@@ -138,11 +138,11 @@ public class SingleDeviceViewOperatorTest {
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath2,
Collections.singleton("sensor1"),
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
true);
@@ -153,7 +153,7 @@ public class SingleDeviceViewOperatorTest {
TimeJoinOperator timeJoinOperator =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
Arrays.asList(seriesScanOperator1, seriesScanOperator2),
Ordering.ASC,
Arrays.asList(TSDataType.INT32, TSDataType.INT32),
@@ -163,7 +163,7 @@ public class SingleDeviceViewOperatorTest {
new AscTimeComparator());
SingleDeviceViewOperator singleDeviceViewOperator =
new SingleDeviceViewOperator(
- fragmentInstanceContext.getOperatorContexts().get(3),
+ driverContext.getOperatorContexts().get(3),
SINGLE_DEVICE_MERGE_OPERATOR_TEST_SG + ".device0",
timeJoinOperator,
Arrays.asList(1, 2),
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/SlidingWindowAggregationOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/SlidingWindowAggregationOperatorTest.java
index c051774df9..863d6bdc9a 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/SlidingWindowAggregationOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/SlidingWindowAggregationOperatorTest.java
@@ -31,6 +31,7 @@ import org.apache.iotdb.db.mpp.aggregation.slidingwindow.SlidingWindowAggregator
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.SlidingWindowAggregationOperator;
@@ -207,12 +208,13 @@ public class SlidingWindowAggregationOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId sourceId = queryId.genPlanNodeId();
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
0, sourceId, SeriesAggregationScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, queryId.genPlanNodeId(), SlidingWindowAggregationOperator.class.getSimpleName());
- fragmentInstanceContext
+ driverContext
.getOperatorContexts()
.forEach(
operatorContext -> {
@@ -232,7 +234,7 @@ public class SlidingWindowAggregationOperatorTest {
sourceId,
d0s0,
Collections.singleton("sensor0"),
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
aggregators,
initTimeRangeIterator(groupByTimeParameter, ascending, true),
null,
@@ -256,7 +258,7 @@ public class SlidingWindowAggregationOperatorTest {
}
return new SlidingWindowAggregationOperator(
- fragmentInstanceContext.getOperatorContexts().get(1),
+ driverContext.getOperatorContexts().get(1),
finalAggregators,
initTimeRangeIterator(groupByTimeParameter, ascending, false),
seriesAggregationScanOperator,
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/TimeJoinOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/TimeJoinOperatorTest.java
index 7ac2ebd6fd..cc332e9a82 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/TimeJoinOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/TimeJoinOperatorTest.java
@@ -27,6 +27,7 @@ import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.join.TimeJoinOperator;
@@ -100,21 +101,20 @@ public class TimeJoinOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(
3, new PlanNodeId("3"), TimeJoinOperator.class.getSimpleName());
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
true);
@@ -127,11 +127,11 @@ public class TimeJoinOperatorTest {
new MeasurementPath(TIME_JOIN_OPERATOR_TEST_SG + ".device0.sensor1", TSDataType.INT32);
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath2,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
true);
@@ -142,7 +142,7 @@ public class TimeJoinOperatorTest {
TimeJoinOperator timeJoinOperator =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
Arrays.asList(seriesScanOperator1, seriesScanOperator2),
Ordering.ASC,
Arrays.asList(TSDataType.INT32, TSDataType.INT32),
@@ -200,24 +200,22 @@ public class TimeJoinOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId3 = new PlanNodeId("3");
- fragmentInstanceContext.addOperatorContext(
- 3, planNodeId3, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(3, planNodeId3, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(
4, new PlanNodeId("4"), TimeJoinOperator.class.getSimpleName());
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
true);
@@ -230,11 +228,11 @@ public class TimeJoinOperatorTest {
new MeasurementPath(TIME_JOIN_OPERATOR_TEST_SG + ".device0.sensor1", TSDataType.INT32);
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath2,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
true);
@@ -248,11 +246,11 @@ public class TimeJoinOperatorTest {
TIME_JOIN_OPERATOR_TEST_SG + ".device0.error_sensor", TSDataType.INT32);
SeriesScanOperator seriesScanOperator3 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(2),
planNodeId3,
measurementPath3,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(2),
null,
null,
true);
@@ -263,7 +261,7 @@ public class TimeJoinOperatorTest {
TimeJoinOperator timeJoinOperator =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(3),
+ driverContext.getOperatorContexts().get(3),
Arrays.asList(seriesScanOperator1, seriesScanOperator2, seriesScanOperator3),
Ordering.ASC,
Arrays.asList(TSDataType.INT32, TSDataType.INT32, TSDataType.INT32),
@@ -324,24 +322,22 @@ public class TimeJoinOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
- 1, planNodeId1, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(1, planNodeId1, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(2, planNodeId2, SeriesScanOperator.class.getSimpleName());
PlanNodeId planNodeId3 = new PlanNodeId("3");
- fragmentInstanceContext.addOperatorContext(
- 3, planNodeId3, SeriesScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(3, planNodeId3, SeriesScanOperator.class.getSimpleName());
+ driverContext.addOperatorContext(
4, new PlanNodeId("4"), TimeJoinOperator.class.getSimpleName());
SeriesScanOperator seriesScanOperator1 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(0),
planNodeId1,
measurementPath1,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(0),
null,
null,
false);
@@ -354,11 +350,11 @@ public class TimeJoinOperatorTest {
new MeasurementPath(TIME_JOIN_OPERATOR_TEST_SG + ".device0.sensor1", TSDataType.INT32);
SeriesScanOperator seriesScanOperator2 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(1),
planNodeId2,
measurementPath2,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(1),
null,
null,
false);
@@ -372,11 +368,11 @@ public class TimeJoinOperatorTest {
TIME_JOIN_OPERATOR_TEST_SG + ".device0.error_sensor", TSDataType.INT32);
SeriesScanOperator seriesScanOperator3 =
new SeriesScanOperator(
+ driverContext.getOperatorContexts().get(2),
planNodeId3,
measurementPath3,
allSensors,
TSDataType.INT32,
- fragmentInstanceContext.getOperatorContexts().get(2),
null,
null,
true);
@@ -387,7 +383,7 @@ public class TimeJoinOperatorTest {
TimeJoinOperator timeJoinOperator =
new TimeJoinOperator(
- fragmentInstanceContext.getOperatorContexts().get(3),
+ driverContext.getOperatorContexts().get(3),
Arrays.asList(seriesScanOperator1, seriesScanOperator2, seriesScanOperator3),
Ordering.DESC,
Arrays.asList(TSDataType.INT32, TSDataType.INT32, TSDataType.INT32),
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/UpdateLastCacheOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/UpdateLastCacheOperatorTest.java
index df07d678b4..569361338f 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/UpdateLastCacheOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/UpdateLastCacheOperatorTest.java
@@ -28,6 +28,7 @@ import org.apache.iotdb.db.mpp.aggregation.Aggregator;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.last.LastQueryUtil;
@@ -192,14 +193,14 @@ public class UpdateLastCacheOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId1, SeriesAggregationScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
- 2, planNodeId2, UpdateLastCacheOperator.class.getSimpleName());
+ driverContext.addOperatorContext(2, planNodeId2, UpdateLastCacheOperator.class.getSimpleName());
- fragmentInstanceContext
+ driverContext
.getOperatorContexts()
.forEach(
operatorContext -> {
@@ -211,7 +212,7 @@ public class UpdateLastCacheOperatorTest {
planNodeId1,
measurementPath,
allSensors,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
aggregators,
initTimeRangeIterator(groupByTimeParameter, ascending, true),
timeFilter,
@@ -222,7 +223,7 @@ public class UpdateLastCacheOperatorTest {
new QueryDataSource(seqResources, unSeqResources));
return new UpdateLastCacheOperator(
- fragmentInstanceContext.getOperatorContexts().get(1),
+ driverContext.getOperatorContexts().get(1),
seriesAggregationScanOperator,
measurementPath,
measurementPath.getSeriesType(),
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/VerticallyConcatOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/VerticallyConcatOperatorTest.java
index 948c2c0779..ed71edf173 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/VerticallyConcatOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/VerticallyConcatOperatorTest.java
@@ -29,6 +29,7 @@ import org.apache.iotdb.db.mpp.aggregation.Aggregator;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
import org.apache.iotdb.db.mpp.execution.operator.process.join.VerticallyConcatOperator;
@@ -102,14 +103,15 @@ public class VerticallyConcatOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId1 = new PlanNodeId("1");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId1, SeriesAggregationScanOperator.class.getSimpleName());
PlanNodeId planNodeId2 = new PlanNodeId("2");
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
2, planNodeId2, SeriesAggregationScanOperator.class.getSimpleName());
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
3, new PlanNodeId("3"), VerticallyConcatOperator.class.getSimpleName());
MeasurementPath measurementPath1 =
@@ -126,7 +128,7 @@ public class VerticallyConcatOperatorTest {
planNodeId1,
measurementPath1,
allSensors,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
aggregators,
initTimeRangeIterator(groupByTimeParameter, true, true),
null,
@@ -147,7 +149,7 @@ public class VerticallyConcatOperatorTest {
planNodeId2,
measurementPath2,
allSensors,
- fragmentInstanceContext.getOperatorContexts().get(1),
+ driverContext.getOperatorContexts().get(1),
aggregators,
initTimeRangeIterator(groupByTimeParameter, true, true),
null,
@@ -162,7 +164,7 @@ public class VerticallyConcatOperatorTest {
VerticallyConcatOperator verticallyConcatOperator =
new VerticallyConcatOperator(
- fragmentInstanceContext.getOperatorContexts().get(2),
+ driverContext.getOperatorContexts().get(2),
Arrays.asList(seriesAggregationScanOperator1, seriesAggregationScanOperator2),
Arrays.asList(
TSDataType.INT64,
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/schema/CountGroupByLevelMergeOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/schema/CountGroupByLevelMergeOperatorTest.java
index da42488e97..f5a50001b6 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/schema/CountGroupByLevelMergeOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/schema/CountGroupByLevelMergeOperatorTest.java
@@ -27,6 +27,7 @@ import org.apache.iotdb.db.metadata.schemaregion.ISchemaRegion;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.driver.SchemaDriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
@@ -68,32 +69,32 @@ public class CountGroupByLevelMergeOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId = queryId.genPlanNodeId();
OperatorContext operatorContext =
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId, CountGroupByLevelScanOperator.class.getSimpleName());
ISchemaRegion schemaRegion = Mockito.mock(ISchemaRegion.class);
- operatorContext
- .getInstanceContext()
- .setDriverContext(new SchemaDriverContext(fragmentInstanceContext, schemaRegion));
+ operatorContext.setDriverContext(
+ new SchemaDriverContext(fragmentInstanceContext, schemaRegion));
CountGroupByLevelScanOperator<ITimeSeriesSchemaInfo> timeSeriesCountOperator1 =
new CountGroupByLevelScanOperator<>(
planNodeId,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
2,
mockSchemaSource(schemaRegion, new PartialPath(OPERATOR_TEST_SG + ".device2")));
CountGroupByLevelScanOperator<ITimeSeriesSchemaInfo> timeSeriesCountOperator2 =
new CountGroupByLevelScanOperator<>(
planNodeId,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
2,
mockSchemaSource(schemaRegion, new PartialPath(OPERATOR_TEST_SG)));
CountGroupByLevelMergeOperator mergeOperator =
new CountGroupByLevelMergeOperator(
planNodeId,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
Arrays.asList(timeSeriesCountOperator1, timeSeriesCountOperator2));
Assert.assertTrue(mergeOperator.isBlocked().isDone());
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaCountOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaCountOperatorTest.java
index 98fac8d15e..c818b729cb 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaCountOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaCountOperatorTest.java
@@ -28,6 +28,7 @@ import org.apache.iotdb.db.metadata.schemaregion.ISchemaRegion;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.driver.SchemaDriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
@@ -65,14 +66,14 @@ public class SchemaCountOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId = queryId.genPlanNodeId();
ISchemaRegion schemaRegion = Mockito.mock(ISchemaRegion.class);
OperatorContext operatorContext =
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId, SchemaCountOperator.class.getSimpleName());
- operatorContext
- .getInstanceContext()
- .setDriverContext(new SchemaDriverContext(fragmentInstanceContext, schemaRegion));
+ operatorContext.setDriverContext(
+ new SchemaDriverContext(fragmentInstanceContext, schemaRegion));
ISchemaSource<?> schemaSource = Mockito.mock(ISchemaSource.class);
List<ISchemaInfo> schemaInfoList = new ArrayList<>(10);
@@ -99,7 +100,7 @@ public class SchemaCountOperatorTest {
SchemaCountOperator<?> schemaCountOperator =
new SchemaCountOperator<>(
- planNodeId, fragmentInstanceContext.getOperatorContexts().get(0), schemaSource);
+ planNodeId, driverContext.getOperatorContexts().get(0), schemaSource);
TsBlock tsBlock = null;
while (schemaCountOperator.hasNext()) {
@@ -124,19 +125,19 @@ public class SchemaCountOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId = queryId.genPlanNodeId();
OperatorContext operatorContext =
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId, CountGroupByLevelScanOperator.class.getSimpleName());
ISchemaRegion schemaRegion = Mockito.mock(ISchemaRegion.class);
- operatorContext
- .getInstanceContext()
- .setDriverContext(new SchemaDriverContext(fragmentInstanceContext, schemaRegion));
+ operatorContext.setDriverContext(
+ new SchemaDriverContext(fragmentInstanceContext, schemaRegion));
CountGroupByLevelScanOperator<ITimeSeriesSchemaInfo> timeSeriesCountOperator =
new CountGroupByLevelScanOperator<>(
planNodeId,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
2,
mockSchemaSource(schemaRegion));
TsBlock tsBlock = null;
@@ -153,7 +154,7 @@ public class SchemaCountOperatorTest {
CountGroupByLevelScanOperator<ITimeSeriesSchemaInfo> timeSeriesCountOperator2 =
new CountGroupByLevelScanOperator<>(
planNodeId,
- fragmentInstanceContext.getOperatorContexts().get(0),
+ driverContext.getOperatorContexts().get(0),
1,
mockSchemaSource(schemaRegion));
tsBlockList = collectResult(timeSeriesCountOperator2);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaQueryScanOperatorTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaQueryScanOperatorTest.java
index 808c800237..9b90fc14c5 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaQueryScanOperatorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/schema/SchemaQueryScanOperatorTest.java
@@ -30,6 +30,7 @@ import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.common.QueryId;
import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
import org.apache.iotdb.db.mpp.common.header.ColumnHeaderConstant;
+import org.apache.iotdb.db.mpp.execution.driver.DriverContext;
import org.apache.iotdb.db.mpp.execution.driver.SchemaDriverContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceStateMachine;
@@ -76,9 +77,10 @@ public class SchemaQueryScanOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId = queryId.genPlanNodeId();
OperatorContext operatorContext =
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId, SchemaQueryScanOperator.class.getSimpleName());
PartialPath partialPath = new PartialPath(META_SCAN_OPERATOR_TEST_SG + ".device0");
ISchemaRegion schemaRegion = Mockito.mock(ISchemaRegion.class);
@@ -88,9 +90,8 @@ public class SchemaQueryScanOperatorTest {
.thenReturn(META_SCAN_OPERATOR_TEST_SG + ".device0");
Mockito.when(deviceSchemaInfo.isAligned()).thenReturn(false);
Iterator<IDeviceSchemaInfo> iterator = Collections.singletonList(deviceSchemaInfo).iterator();
- operatorContext
- .getInstanceContext()
- .setDriverContext(new SchemaDriverContext(fragmentInstanceContext, schemaRegion));
+ operatorContext.setDriverContext(
+ new SchemaDriverContext(fragmentInstanceContext, schemaRegion));
ISchemaSource<IDeviceSchemaInfo> deviceSchemaSource =
SchemaSourceFactory.getDeviceSchemaSource(partialPath, false, 10, 0, true);
Mockito.when(deviceSchemaSource.getSchemaReader(schemaRegion))
@@ -109,11 +110,12 @@ public class SchemaQueryScanOperatorTest {
return iterator.next();
}
});
+
List<ColumnHeader> columns = deviceSchemaSource.getInfoQueryColumnHeaders();
SchemaQueryScanOperator<IDeviceSchemaInfo> devicesSchemaScanOperator =
new SchemaQueryScanOperator<>(
- planNodeId, fragmentInstanceContext.getOperatorContexts().get(0), deviceSchemaSource);
+ planNodeId, driverContext.getOperatorContexts().get(0), deviceSchemaSource);
while (devicesSchemaScanOperator.hasNext()) {
TsBlock tsBlock = devicesSchemaScanOperator.next();
@@ -162,9 +164,10 @@ public class SchemaQueryScanOperatorTest {
new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor);
FragmentInstanceContext fragmentInstanceContext =
createFragmentInstanceContext(instanceId, stateMachine);
+ DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0);
PlanNodeId planNodeId = queryId.genPlanNodeId();
OperatorContext operatorContext =
- fragmentInstanceContext.addOperatorContext(
+ driverContext.addOperatorContext(
1, planNodeId, SchemaQueryScanOperator.class.getSimpleName());
PartialPath partialPath = new PartialPath(META_SCAN_OPERATOR_TEST_SG + ".device0.*");
@@ -186,9 +189,8 @@ public class SchemaQueryScanOperatorTest {
ISchemaRegion schemaRegion = Mockito.mock(ISchemaRegion.class);
Mockito.when(schemaRegion.getStorageGroupFullPath()).thenReturn(META_SCAN_OPERATOR_TEST_SG);
- operatorContext
- .getInstanceContext()
- .setDriverContext(new SchemaDriverContext(fragmentInstanceContext, schemaRegion));
+ operatorContext.setDriverContext(
+ new SchemaDriverContext(fragmentInstanceContext, schemaRegion));
ISchemaSource<ITimeSeriesSchemaInfo> timeSeriesSchemaSource =
SchemaSourceFactory.getTimeSeriesSchemaSource(
partialPath, false, 10, 0, null, null, false, Collections.emptyMap());
@@ -208,11 +210,10 @@ public class SchemaQueryScanOperatorTest {
return iterator.next();
}
});
+
SchemaQueryScanOperator<ITimeSeriesSchemaInfo> timeSeriesMetaScanOperator =
new SchemaQueryScanOperator<>(
- planNodeId,
- fragmentInstanceContext.getOperatorContexts().get(0),
- timeSeriesSchemaSource);
+ planNodeId, driverContext.getOperatorContexts().get(0), timeSeriesSchemaSource);
while (timeSeriesMetaScanOperator.hasNext()) {
TsBlock tsBlock = timeSeriesMetaScanOperator.next();
assertEquals(
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/DefaultDriverSchedulerTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/DefaultDriverSchedulerTest.java
index e7e6861faa..8a3d5280b3 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/DefaultDriverSchedulerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/DefaultDriverSchedulerTest.java
@@ -24,6 +24,7 @@ import org.apache.iotdb.db.mpp.common.QueryId;
import org.apache.iotdb.db.mpp.execution.driver.IDriver;
import org.apache.iotdb.db.mpp.execution.exchange.IMPPDataExchangeManager;
import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTask;
+import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTaskId;
import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTaskStatus;
import org.apache.iotdb.db.utils.stats.CpuTimer;
import org.apache.iotdb.mpp.rpc.thrift.IDataNodeRPCService;
@@ -36,7 +37,9 @@ import org.mockito.Mockito;
import java.io.IOException;
import java.util.HashSet;
+import java.util.Map;
import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
public class DefaultDriverSchedulerTest {
@@ -58,7 +61,8 @@ public class DefaultDriverSchedulerTest {
QueryId queryId = new QueryId("test");
FragmentInstanceId instanceId =
new FragmentInstanceId(new PlanFragmentId(queryId, 0), "inst-0");
- Mockito.when(mockDriver.getInfo()).thenReturn(instanceId);
+ DriverTaskId driverTaskID = new DriverTaskId(instanceId, 0);
+ Mockito.when(mockDriver.getDriverTaskId()).thenReturn(driverTaskID);
DriverTaskStatus[] invalidStates =
new DriverTaskStatus[] {
DriverTaskStatus.FINISHED,
@@ -71,30 +75,34 @@ public class DefaultDriverSchedulerTest {
manager.getBlockedTasks().add(testTask);
Set<DriverTask> taskSet = new HashSet<>();
taskSet.add(testTask);
- manager.getQueryMap().put(queryId, taskSet);
+ Map<FragmentInstanceId, Set<DriverTask>> fragmentRelatedTask = new ConcurrentHashMap<>();
+ fragmentRelatedTask.put(instanceId, taskSet);
+ manager.getQueryMap().put(queryId, fragmentRelatedTask);
manager.getTimeoutQueue().push(testTask);
defaultScheduler.blockedToReady(testTask);
Assert.assertEquals(status, testTask.getStatus());
Assert.assertTrue(manager.getBlockedTasks().contains(testTask));
- Assert.assertNull(manager.getReadyQueue().get(testTask.getId()));
- Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getId()));
+ Assert.assertNull(manager.getReadyQueue().get(testTask.getDriverTaskId()));
+ Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getDriverTaskId()));
Assert.assertTrue(manager.getQueryMap().containsKey(queryId));
- Assert.assertTrue(manager.getQueryMap().get(queryId).contains(testTask));
+ Assert.assertTrue(manager.getQueryMap().get(queryId).get(instanceId).contains(testTask));
clear();
}
DriverTask testTask = new DriverTask(mockDriver, 100L, DriverTaskStatus.BLOCKED);
manager.getBlockedTasks().add(testTask);
Set<DriverTask> taskSet = new HashSet<>();
taskSet.add(testTask);
- manager.getQueryMap().put(queryId, taskSet);
+ Map<FragmentInstanceId, Set<DriverTask>> fragmentRelatedTask = new ConcurrentHashMap<>();
+ fragmentRelatedTask.put(instanceId, taskSet);
+ manager.getQueryMap().put(queryId, fragmentRelatedTask);
manager.getTimeoutQueue().push(testTask);
defaultScheduler.blockedToReady(testTask);
Assert.assertEquals(DriverTaskStatus.READY, testTask.getStatus());
Assert.assertFalse(manager.getBlockedTasks().contains(testTask));
- Assert.assertNotNull(manager.getReadyQueue().get(testTask.getId()));
- Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getId()));
+ Assert.assertNotNull(manager.getReadyQueue().get(testTask.getDriverTaskId()));
+ Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getDriverTaskId()));
Assert.assertTrue(manager.getQueryMap().containsKey(queryId));
- Assert.assertTrue(manager.getQueryMap().get(queryId).contains(testTask));
+ Assert.assertTrue(manager.getQueryMap().get(queryId).get(instanceId).contains(testTask));
Mockito.verify(mockDriver, Mockito.never()).failed(Mockito.any());
clear();
}
@@ -110,7 +118,8 @@ public class DefaultDriverSchedulerTest {
QueryId queryId = new QueryId("test");
FragmentInstanceId instanceId =
new FragmentInstanceId(new PlanFragmentId(queryId, 0), "inst-0");
- Mockito.when(mockDriver.getInfo()).thenReturn(instanceId);
+ DriverTaskId driverTaskID = new DriverTaskId(instanceId, 0);
+ Mockito.when(mockDriver.getDriverTaskId()).thenReturn(driverTaskID);
DriverTaskStatus[] invalidStates =
new DriverTaskStatus[] {
DriverTaskStatus.FINISHED,
@@ -122,27 +131,31 @@ public class DefaultDriverSchedulerTest {
DriverTask testTask = new DriverTask(mockDriver, 100L, status);
Set<DriverTask> taskSet = new HashSet<>();
taskSet.add(testTask);
- manager.getQueryMap().put(queryId, taskSet);
+ Map<FragmentInstanceId, Set<DriverTask>> fragmentRelatedTask = new ConcurrentHashMap<>();
+ fragmentRelatedTask.put(instanceId, taskSet);
+ manager.getQueryMap().put(queryId, fragmentRelatedTask);
manager.getTimeoutQueue().push(testTask);
defaultScheduler.readyToRunning(testTask);
Assert.assertEquals(status, testTask.getStatus());
Assert.assertFalse(manager.getBlockedTasks().contains(testTask));
- Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getId()));
+ Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getDriverTaskId()));
Assert.assertTrue(manager.getQueryMap().containsKey(queryId));
- Assert.assertTrue(manager.getQueryMap().get(queryId).contains(testTask));
+ Assert.assertTrue(manager.getQueryMap().get(queryId).get(instanceId).contains(testTask));
clear();
}
DriverTask testTask = new DriverTask(mockDriver, 100L, DriverTaskStatus.READY);
Set<DriverTask> taskSet = new HashSet<>();
taskSet.add(testTask);
- manager.getQueryMap().put(queryId, taskSet);
+ Map<FragmentInstanceId, Set<DriverTask>> fragmentRelatedTask = new ConcurrentHashMap<>();
+ fragmentRelatedTask.put(instanceId, taskSet);
+ manager.getQueryMap().put(queryId, fragmentRelatedTask);
manager.getTimeoutQueue().push(testTask);
defaultScheduler.readyToRunning(testTask);
Assert.assertEquals(DriverTaskStatus.RUNNING, testTask.getStatus());
Assert.assertFalse(manager.getBlockedTasks().contains(testTask));
- Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getId()));
+ Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getDriverTaskId()));
Assert.assertTrue(manager.getQueryMap().containsKey(queryId));
- Assert.assertTrue(manager.getQueryMap().get(queryId).contains(testTask));
+ Assert.assertTrue(manager.getQueryMap().get(queryId).get(instanceId).contains(testTask));
Mockito.verify(mockDriver, Mockito.never()).failed(Mockito.any());
clear();
}
@@ -157,7 +170,8 @@ public class DefaultDriverSchedulerTest {
QueryId queryId = new QueryId("test");
FragmentInstanceId instanceId =
new FragmentInstanceId(new PlanFragmentId(queryId, 0), "inst-0");
- Mockito.when(mockDriver.getInfo()).thenReturn(instanceId);
+ DriverTaskId driverTaskID = new DriverTaskId(instanceId, 0);
+ Mockito.when(mockDriver.getDriverTaskId()).thenReturn(driverTaskID);
DriverTaskStatus[] invalidStates =
new DriverTaskStatus[] {
DriverTaskStatus.FINISHED,
@@ -169,21 +183,25 @@ public class DefaultDriverSchedulerTest {
DriverTask testTask = new DriverTask(mockDriver, 100L, status);
Set<DriverTask> taskSet = new HashSet<>();
taskSet.add(testTask);
- manager.getQueryMap().put(queryId, taskSet);
+ Map<FragmentInstanceId, Set<DriverTask>> fragmentRelatedTask = new ConcurrentHashMap<>();
+ fragmentRelatedTask.put(instanceId, taskSet);
+ manager.getQueryMap().put(queryId, fragmentRelatedTask);
manager.getTimeoutQueue().push(testTask);
defaultScheduler.runningToReady(testTask, new ExecutionContext());
Assert.assertEquals(status, testTask.getStatus());
Assert.assertFalse(manager.getBlockedTasks().contains(testTask));
- Assert.assertNull(manager.getReadyQueue().get(testTask.getId()));
- Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getId()));
+ Assert.assertNull(manager.getReadyQueue().get(testTask.getDriverTaskId()));
+ Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getDriverTaskId()));
Assert.assertTrue(manager.getQueryMap().containsKey(queryId));
- Assert.assertTrue(manager.getQueryMap().get(queryId).contains(testTask));
+ Assert.assertTrue(manager.getQueryMap().get(queryId).get(instanceId).contains(testTask));
clear();
}
DriverTask testTask = new DriverTask(mockDriver, 100L, DriverTaskStatus.RUNNING);
Set<DriverTask> taskSet = new HashSet<>();
taskSet.add(testTask);
- manager.getQueryMap().put(queryId, taskSet);
+ Map<FragmentInstanceId, Set<DriverTask>> fragmentRelatedTask = new ConcurrentHashMap<>();
+ fragmentRelatedTask.put(instanceId, taskSet);
+ manager.getQueryMap().put(queryId, fragmentRelatedTask);
manager.getTimeoutQueue().push(testTask);
ExecutionContext context = new ExecutionContext();
context.setTimeSlice(new Duration(1, TimeUnit.SECONDS));
@@ -192,10 +210,10 @@ public class DefaultDriverSchedulerTest {
Assert.assertEquals(0.0D, testTask.getSchedulePriority(), 0.00001);
Assert.assertEquals(DriverTaskStatus.READY, testTask.getStatus());
Assert.assertFalse(manager.getBlockedTasks().contains(testTask));
- Assert.assertNotNull(manager.getReadyQueue().get(testTask.getId()));
- Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getId()));
+ Assert.assertNotNull(manager.getReadyQueue().get(testTask.getDriverTaskId()));
+ Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getDriverTaskId()));
Assert.assertTrue(manager.getQueryMap().containsKey(queryId));
- Assert.assertTrue(manager.getQueryMap().get(queryId).contains(testTask));
+ Assert.assertTrue(manager.getQueryMap().get(queryId).get(instanceId).contains(testTask));
Mockito.verify(mockDriver, Mockito.never()).failed(Mockito.any());
clear();
}
@@ -210,7 +228,8 @@ public class DefaultDriverSchedulerTest {
QueryId queryId = new QueryId("test");
FragmentInstanceId instanceId =
new FragmentInstanceId(new PlanFragmentId(queryId, 0), "inst-0");
- Mockito.when(mockDriver.getInfo()).thenReturn(instanceId);
+ DriverTaskId driverTaskID = new DriverTaskId(instanceId, 0);
+ Mockito.when(mockDriver.getDriverTaskId()).thenReturn(driverTaskID);
DriverTaskStatus[] invalidStates =
new DriverTaskStatus[] {
DriverTaskStatus.FINISHED,
@@ -222,21 +241,25 @@ public class DefaultDriverSchedulerTest {
DriverTask testTask = new DriverTask(mockDriver, 100L, status);
Set<DriverTask> taskSet = new HashSet<>();
taskSet.add(testTask);
- manager.getQueryMap().put(queryId, taskSet);
+ Map<FragmentInstanceId, Set<DriverTask>> fragmentRelatedTask = new ConcurrentHashMap<>();
+ fragmentRelatedTask.put(instanceId, taskSet);
+ manager.getQueryMap().put(queryId, fragmentRelatedTask);
manager.getTimeoutQueue().push(testTask);
defaultScheduler.runningToBlocked(testTask, new ExecutionContext());
Assert.assertEquals(status, testTask.getStatus());
Assert.assertFalse(manager.getBlockedTasks().contains(testTask));
- Assert.assertNull(manager.getReadyQueue().get(testTask.getId()));
- Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getId()));
+ Assert.assertNull(manager.getReadyQueue().get(testTask.getDriverTaskId()));
+ Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getDriverTaskId()));
Assert.assertTrue(manager.getQueryMap().containsKey(queryId));
- Assert.assertTrue(manager.getQueryMap().get(queryId).contains(testTask));
+ Assert.assertTrue(manager.getQueryMap().get(queryId).get(instanceId).contains(testTask));
clear();
}
DriverTask testTask = new DriverTask(mockDriver, 100L, DriverTaskStatus.RUNNING);
Set<DriverTask> taskSet = new HashSet<>();
taskSet.add(testTask);
- manager.getQueryMap().put(queryId, taskSet);
+ Map<FragmentInstanceId, Set<DriverTask>> fragmentRelatedTask = new ConcurrentHashMap<>();
+ fragmentRelatedTask.put(instanceId, taskSet);
+ manager.getQueryMap().put(queryId, fragmentRelatedTask);
manager.getTimeoutQueue().push(testTask);
ExecutionContext context = new ExecutionContext();
context.setTimeSlice(new Duration(1, TimeUnit.SECONDS));
@@ -245,10 +268,10 @@ public class DefaultDriverSchedulerTest {
Assert.assertEquals(0.0D, testTask.getSchedulePriority(), 0.00001);
Assert.assertEquals(DriverTaskStatus.BLOCKED, testTask.getStatus());
Assert.assertTrue(manager.getBlockedTasks().contains(testTask));
- Assert.assertNull(manager.getReadyQueue().get(testTask.getId()));
- Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getId()));
+ Assert.assertNull(manager.getReadyQueue().get(testTask.getDriverTaskId()));
+ Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getDriverTaskId()));
Assert.assertTrue(manager.getQueryMap().containsKey(queryId));
- Assert.assertTrue(manager.getQueryMap().get(queryId).contains(testTask));
+ Assert.assertTrue(manager.getQueryMap().get(queryId).get(instanceId).contains(testTask));
Mockito.verify(mockDriver, Mockito.never()).failed(Mockito.any());
clear();
}
@@ -263,7 +286,8 @@ public class DefaultDriverSchedulerTest {
QueryId queryId = new QueryId("test");
FragmentInstanceId instanceId =
new FragmentInstanceId(new PlanFragmentId(queryId, 0), "inst-0");
- Mockito.when(mockDriver.getInfo()).thenReturn(instanceId);
+ DriverTaskId driverTaskID = new DriverTaskId(instanceId, 0);
+ Mockito.when(mockDriver.getDriverTaskId()).thenReturn(driverTaskID);
DriverTaskStatus[] invalidStates =
new DriverTaskStatus[] {
DriverTaskStatus.FINISHED,
@@ -275,21 +299,25 @@ public class DefaultDriverSchedulerTest {
DriverTask testTask = new DriverTask(mockDriver, 100L, status);
Set<DriverTask> taskSet = new HashSet<>();
taskSet.add(testTask);
- manager.getQueryMap().put(queryId, taskSet);
+ Map<FragmentInstanceId, Set<DriverTask>> fragmentRelatedTask = new ConcurrentHashMap<>();
+ fragmentRelatedTask.put(instanceId, taskSet);
+ manager.getQueryMap().put(queryId, fragmentRelatedTask);
manager.getTimeoutQueue().push(testTask);
defaultScheduler.runningToFinished(testTask, new ExecutionContext());
Assert.assertEquals(status, testTask.getStatus());
Assert.assertFalse(manager.getBlockedTasks().contains(testTask));
- Assert.assertNull(manager.getReadyQueue().get(testTask.getId()));
- Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getId()));
+ Assert.assertNull(manager.getReadyQueue().get(testTask.getDriverTaskId()));
+ Assert.assertNotNull(manager.getTimeoutQueue().get(testTask.getDriverTaskId()));
Assert.assertTrue(manager.getQueryMap().containsKey(queryId));
- Assert.assertTrue(manager.getQueryMap().get(queryId).contains(testTask));
+ Assert.assertTrue(manager.getQueryMap().get(queryId).get(instanceId).contains(testTask));
clear();
}
DriverTask testTask = new DriverTask(mockDriver, 100L, DriverTaskStatus.RUNNING);
Set<DriverTask> taskSet = new HashSet<>();
taskSet.add(testTask);
- manager.getQueryMap().put(queryId, taskSet);
+ Map<FragmentInstanceId, Set<DriverTask>> fragmentRelatedTask = new ConcurrentHashMap<>();
+ fragmentRelatedTask.put(instanceId, taskSet);
+ manager.getQueryMap().put(queryId, fragmentRelatedTask);
manager.getTimeoutQueue().push(testTask);
ExecutionContext context = new ExecutionContext();
context.setTimeSlice(new Duration(1, TimeUnit.SECONDS));
@@ -298,8 +326,8 @@ public class DefaultDriverSchedulerTest {
Assert.assertEquals(0.0D, testTask.getSchedulePriority(), 0.00001);
Assert.assertEquals(DriverTaskStatus.FINISHED, testTask.getStatus());
Assert.assertFalse(manager.getBlockedTasks().contains(testTask));
- Assert.assertNull(manager.getReadyQueue().get(testTask.getId()));
- Assert.assertNull(manager.getTimeoutQueue().get(testTask.getId()));
+ Assert.assertNull(manager.getReadyQueue().get(testTask.getDriverTaskId()));
+ Assert.assertNull(manager.getTimeoutQueue().get(testTask.getDriverTaskId()));
Assert.assertFalse(manager.getQueryMap().containsKey(queryId));
Mockito.verify(mockDriver, Mockito.never()).failed(Mockito.any());
clear();
@@ -316,12 +344,14 @@ public class DefaultDriverSchedulerTest {
QueryId queryId = new QueryId("test");
FragmentInstanceId instanceId1 =
new FragmentInstanceId(new PlanFragmentId(queryId, 0), "inst-0");
+ DriverTaskId driverTaskId1 = new DriverTaskId(instanceId1, 0);
IDriver mockDriver1 = Mockito.mock(IDriver.class);
- Mockito.when(mockDriver1.getInfo()).thenReturn(instanceId1);
+ Mockito.when(mockDriver1.getDriverTaskId()).thenReturn(driverTaskId1);
IDriver mockDriver2 = Mockito.mock(IDriver.class);
FragmentInstanceId instanceId2 =
new FragmentInstanceId(new PlanFragmentId(queryId, 0), "inst-1");
- Mockito.when(mockDriver2.getInfo()).thenReturn(instanceId2);
+ DriverTaskId driverTaskId2 = new DriverTaskId(instanceId2, 0);
+ Mockito.when(mockDriver2.getDriverTaskId()).thenReturn(driverTaskId2);
DriverTaskStatus[] invalidStates =
new DriverTaskStatus[] {
DriverTaskStatus.FINISHED, DriverTaskStatus.ABORTED,
@@ -329,10 +359,14 @@ public class DefaultDriverSchedulerTest {
for (DriverTaskStatus status : invalidStates) {
DriverTask testTask1 = new DriverTask(mockDriver1, 100L, status);
DriverTask testTask2 = new DriverTask(mockDriver2, 100L, DriverTaskStatus.BLOCKED);
- Set<DriverTask> taskSet = new HashSet<>();
- taskSet.add(testTask1);
- taskSet.add(testTask2);
- manager.getQueryMap().put(queryId, taskSet);
+ Set<DriverTask> taskSet1 = new HashSet<>();
+ taskSet1.add(testTask1);
+ Set<DriverTask> taskSet2 = new HashSet<>();
+ taskSet2.add(testTask2);
+ Map<FragmentInstanceId, Set<DriverTask>> fragmentRelatedTask = new ConcurrentHashMap<>();
+ fragmentRelatedTask.put(instanceId1, taskSet1);
+ fragmentRelatedTask.put(instanceId2, taskSet2);
+ manager.getQueryMap().put(queryId, fragmentRelatedTask);
manager.getTimeoutQueue().push(testTask1);
manager.getTimeoutQueue().push(testTask2);
manager.getBlockedTasks().add(testTask2);
@@ -342,13 +376,13 @@ public class DefaultDriverSchedulerTest {
Assert.assertEquals(DriverTaskStatus.BLOCKED, testTask2.getStatus());
Assert.assertFalse(manager.getBlockedTasks().contains(testTask1));
Assert.assertTrue(manager.getBlockedTasks().contains(testTask2));
- Assert.assertNull(manager.getReadyQueue().get(testTask1.getId()));
- Assert.assertNull(manager.getReadyQueue().get(testTask2.getId()));
- Assert.assertNotNull(manager.getTimeoutQueue().get(testTask1.getId()));
- Assert.assertNotNull(manager.getTimeoutQueue().get(testTask2.getId()));
+ Assert.assertNull(manager.getReadyQueue().get(testTask1.getDriverTaskId()));
+ Assert.assertNull(manager.getReadyQueue().get(testTask2.getDriverTaskId()));
+ Assert.assertNotNull(manager.getTimeoutQueue().get(testTask1.getDriverTaskId()));
+ Assert.assertNotNull(manager.getTimeoutQueue().get(testTask2.getDriverTaskId()));
Assert.assertTrue(manager.getQueryMap().containsKey(queryId));
- Assert.assertTrue(manager.getQueryMap().get(queryId).contains(testTask1));
- Assert.assertTrue(manager.getQueryMap().get(queryId).contains(testTask2));
+ Assert.assertTrue(manager.getQueryMap().get(queryId).get(instanceId1).contains(testTask1));
+ Assert.assertTrue(manager.getQueryMap().get(queryId).get(instanceId2).contains(testTask2));
Mockito.verify(mockDriver1, Mockito.never()).failed(Mockito.any());
Mockito.verify(mockDriver2, Mockito.never()).failed(Mockito.any());
@@ -360,17 +394,21 @@ public class DefaultDriverSchedulerTest {
};
for (DriverTaskStatus status : validStates) {
Mockito.reset(mockDriver1);
- Mockito.when(mockDriver1.getInfo()).thenReturn(instanceId1);
+ Mockito.when(mockDriver1.getDriverTaskId()).thenReturn(driverTaskId1);
Mockito.reset(mockDriver2);
- Mockito.when(mockDriver2.getInfo()).thenReturn(instanceId2);
+ Mockito.when(mockDriver2.getDriverTaskId()).thenReturn(driverTaskId2);
DriverTask testTask1 = new DriverTask(mockDriver1, 100L, status);
DriverTask testTask2 = new DriverTask(mockDriver2, 100L, DriverTaskStatus.BLOCKED);
- Set<DriverTask> taskSet = new HashSet<>();
- taskSet.add(testTask1);
- taskSet.add(testTask2);
- manager.getQueryMap().put(queryId, taskSet);
+ Set<DriverTask> taskSet1 = new HashSet<>();
+ taskSet1.add(testTask1);
+ Set<DriverTask> taskSet2 = new HashSet<>();
+ taskSet2.add(testTask2);
+ Map<FragmentInstanceId, Set<DriverTask>> fragmentRelatedTask = new ConcurrentHashMap<>();
+ fragmentRelatedTask.put(instanceId1, taskSet1);
+ fragmentRelatedTask.put(instanceId2, taskSet2);
+ manager.getQueryMap().put(queryId, fragmentRelatedTask);
manager.getTimeoutQueue().push(testTask1);
defaultScheduler.toAborted(testTask1);
@@ -384,10 +422,10 @@ public class DefaultDriverSchedulerTest {
Assert.assertEquals(DriverTaskStatus.ABORTED, testTask2.getStatus());
Assert.assertFalse(manager.getBlockedTasks().contains(testTask1));
Assert.assertFalse(manager.getBlockedTasks().contains(testTask2));
- Assert.assertNull(manager.getReadyQueue().get(testTask1.getId()));
- Assert.assertNull(manager.getReadyQueue().get(testTask2.getId()));
- Assert.assertNull(manager.getTimeoutQueue().get(testTask1.getId()));
- Assert.assertNull(manager.getTimeoutQueue().get(testTask2.getId()));
+ Assert.assertNull(manager.getReadyQueue().get(testTask1.getDriverTaskId()));
+ Assert.assertNull(manager.getReadyQueue().get(testTask2.getDriverTaskId()));
+ Assert.assertNull(manager.getTimeoutQueue().get(testTask1.getDriverTaskId()));
+ Assert.assertNull(manager.getTimeoutQueue().get(testTask2.getDriverTaskId()));
Assert.assertFalse(manager.getQueryMap().containsKey(queryId));
// The mockDriver1.failed() will be called outside the scheduler
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/DriverSchedulerTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/DriverSchedulerTest.java
index b40339d80a..757f39358e 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/DriverSchedulerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/DriverSchedulerTest.java
@@ -25,7 +25,7 @@ import org.apache.iotdb.db.mpp.common.QueryId;
import org.apache.iotdb.db.mpp.execution.driver.IDriver;
import org.apache.iotdb.db.mpp.execution.exchange.IMPPDataExchangeManager;
import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTask;
-import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTaskID;
+import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTaskId;
import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTaskStatus;
import org.junit.After;
@@ -60,11 +60,13 @@ public class DriverSchedulerTest {
QueryId queryId = new QueryId("test");
PlanFragmentId fragmentId = new PlanFragmentId(queryId, 0);
FragmentInstanceId instanceId1 = new FragmentInstanceId(fragmentId, "inst-0");
+ DriverTaskId driverTaskId1 = new DriverTaskId(instanceId1, 0);
IDriver mockDriver1 = Mockito.mock(IDriver.class);
- Mockito.when(mockDriver1.getInfo()).thenReturn(instanceId1);
+ Mockito.when(mockDriver1.getDriverTaskId()).thenReturn(driverTaskId1);
FragmentInstanceId instanceId2 = new FragmentInstanceId(fragmentId, "inst-1");
+ DriverTaskId driverTaskId2 = new DriverTaskId(instanceId2, 0);
IDriver mockDriver2 = Mockito.mock(IDriver.class);
- Mockito.when(mockDriver2.getInfo()).thenReturn(instanceId2);
+ Mockito.when(mockDriver2.getDriverTaskId()).thenReturn(driverTaskId2);
List<IDriver> instances = Arrays.asList(mockDriver1, mockDriver2);
manager.submitDrivers(queryId, instances, QUERY_TIMEOUT_MS);
Assert.assertTrue(manager.getBlockedTasks().isEmpty());
@@ -73,19 +75,20 @@ public class DriverSchedulerTest {
Assert.assertEquals(2, manager.getQueryMap().get(queryId).size());
Assert.assertEquals(2, manager.getTimeoutQueue().size());
Assert.assertEquals(2, manager.getReadyQueue().size());
- DriverTask task1 = manager.getTimeoutQueue().get(new DriverTaskID(instanceId1));
+ DriverTask task1 = manager.getTimeoutQueue().get(driverTaskId1);
Assert.assertNotNull(task1);
- DriverTask task2 = manager.getTimeoutQueue().get(new DriverTaskID(instanceId2));
+ DriverTask task2 = manager.getTimeoutQueue().get(driverTaskId2);
Assert.assertNotNull(task2);
- Assert.assertTrue(manager.getQueryMap().get(queryId).contains(task1));
- Assert.assertTrue(manager.getQueryMap().get(queryId).contains(task2));
+ Assert.assertTrue(manager.getQueryMap().get(queryId).get(instanceId1).contains(task1));
+ Assert.assertTrue(manager.getQueryMap().get(queryId).get(instanceId2).contains(task2));
Assert.assertEquals(DriverTaskStatus.READY, task1.getStatus());
Assert.assertEquals(DriverTaskStatus.READY, task2.getStatus());
// Submit another task of the same query
IDriver mockDriver3 = Mockito.mock(IDriver.class);
FragmentInstanceId instanceId3 = new FragmentInstanceId(fragmentId, "inst-2");
- Mockito.when(mockDriver3.getInfo()).thenReturn(instanceId3);
+ DriverTaskId driverTaskId3 = new DriverTaskId(instanceId3, 0);
+ Mockito.when(mockDriver3.getDriverTaskId()).thenReturn(driverTaskId3);
manager.submitDrivers(queryId, Collections.singletonList(mockDriver3), QUERY_TIMEOUT_MS);
Assert.assertTrue(manager.getBlockedTasks().isEmpty());
Assert.assertEquals(1, manager.getQueryMap().size());
@@ -93,17 +96,18 @@ public class DriverSchedulerTest {
Assert.assertEquals(3, manager.getQueryMap().get(queryId).size());
Assert.assertEquals(3, manager.getTimeoutQueue().size());
Assert.assertEquals(3, manager.getReadyQueue().size());
- DriverTask task3 = manager.getTimeoutQueue().get(new DriverTaskID(instanceId3));
+ DriverTask task3 = manager.getTimeoutQueue().get(driverTaskId3);
Assert.assertNotNull(task3);
- Assert.assertTrue(manager.getQueryMap().get(queryId).contains(task3));
+ Assert.assertTrue(manager.getQueryMap().get(queryId).get(instanceId3).contains(task3));
Assert.assertEquals(DriverTaskStatus.READY, task3.getStatus());
// Submit another task of the different query
QueryId queryId2 = new QueryId("test2");
PlanFragmentId fragmentId2 = new PlanFragmentId(queryId2, 0);
FragmentInstanceId instanceId4 = new FragmentInstanceId(fragmentId2, "inst-0");
+ DriverTaskId driverTaskId4 = new DriverTaskId(instanceId4, 0);
IDriver mockDriver4 = Mockito.mock(IDriver.class);
- Mockito.when(mockDriver4.getInfo()).thenReturn(instanceId4);
+ Mockito.when(mockDriver4.getDriverTaskId()).thenReturn(driverTaskId4);
manager.submitDrivers(queryId2, Collections.singletonList(mockDriver4), QUERY_TIMEOUT_MS);
Assert.assertTrue(manager.getBlockedTasks().isEmpty());
Assert.assertEquals(2, manager.getQueryMap().size());
@@ -111,14 +115,14 @@ public class DriverSchedulerTest {
Assert.assertEquals(1, manager.getQueryMap().get(queryId2).size());
Assert.assertEquals(4, manager.getTimeoutQueue().size());
Assert.assertEquals(4, manager.getReadyQueue().size());
- DriverTask task4 = manager.getTimeoutQueue().get(new DriverTaskID(instanceId4));
+ DriverTask task4 = manager.getTimeoutQueue().get(driverTaskId4);
Assert.assertNotNull(task4);
- Assert.assertTrue(manager.getQueryMap().get(queryId2).contains(task4));
+ Assert.assertTrue(manager.getQueryMap().get(queryId2).get(instanceId4).contains(task4));
Assert.assertEquals(DriverTaskStatus.READY, task4.getStatus());
// Abort one FragmentInstance
Mockito.reset(mockDriver1);
- Mockito.when(mockDriver1.getInfo()).thenReturn(instanceId1);
+ Mockito.when(mockDriver1.getDriverTaskId()).thenReturn(driverTaskId1);
manager.abortFragmentInstance(instanceId1);
Mockito.verify(mockMPPDataExchangeManager, Mockito.times(1))
.forceDeregisterFragmentInstance(Mockito.any());
@@ -132,17 +136,16 @@ public class DriverSchedulerTest {
Assert.assertEquals(DriverTaskStatus.READY, task3.getStatus());
Assert.assertEquals(DriverTaskStatus.READY, task4.getStatus());
Mockito.verify(mockDriver1, Mockito.times(1)).failed(Mockito.any());
- Assert.assertEquals(
- FragmentInstanceAbortedException.BY_FRAGMENT_ABORT_CALLED, task1.getAbortCause());
+ Assert.assertEquals(DriverTaskAbortedException.BY_FRAGMENT_ABORT_CALLED, task1.getAbortCause());
// Abort the whole query
Mockito.reset(mockMPPDataExchangeManager);
Mockito.reset(mockDriver1);
- Mockito.when(mockDriver1.getInfo()).thenReturn(instanceId1);
+ Mockito.when(mockDriver1.getDriverTaskId()).thenReturn(driverTaskId1);
Mockito.reset(mockDriver2);
- Mockito.when(mockDriver2.getInfo()).thenReturn(instanceId2);
+ Mockito.when(mockDriver2.getDriverTaskId()).thenReturn(driverTaskId2);
Mockito.reset(mockDriver3);
- Mockito.when(mockDriver3.getInfo()).thenReturn(instanceId3);
+ Mockito.when(mockDriver3.getDriverTaskId()).thenReturn(driverTaskId3);
manager.abortQuery(queryId);
Mockito.verify(mockMPPDataExchangeManager, Mockito.times(2))
.forceDeregisterFragmentInstance(Mockito.any());
@@ -160,9 +163,9 @@ public class DriverSchedulerTest {
Mockito.verify(mockDriver3, Mockito.times(1)).failed(Mockito.any());
Mockito.verify(mockDriver4, Mockito.never()).failed(Mockito.any());
Assert.assertEquals(
- FragmentInstanceAbortedException.BY_QUERY_CASCADING_ABORTED, task2.getAbortCause());
+ DriverTaskAbortedException.BY_QUERY_CASCADING_ABORTED, task2.getAbortCause());
Assert.assertEquals(
- FragmentInstanceAbortedException.BY_QUERY_CASCADING_ABORTED, task3.getAbortCause());
+ DriverTaskAbortedException.BY_QUERY_CASCADING_ABORTED, task3.getAbortCause());
Assert.assertNull(task4.getAbortCause());
}
}
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskTimeoutSentinelThreadTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskTimeoutSentinelThreadTest.java
index d61768e849..d55b4ead86 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskTimeoutSentinelThreadTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskTimeoutSentinelThreadTest.java
@@ -26,6 +26,7 @@ import org.apache.iotdb.db.mpp.execution.schedule.queue.IndexedBlockingQueue;
import org.apache.iotdb.db.mpp.execution.schedule.queue.L1PriorityQueue;
import org.apache.iotdb.db.mpp.execution.schedule.queue.L2PriorityQueue;
import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTask;
+import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTaskId;
import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTaskStatus;
import com.google.common.util.concurrent.Futures;
@@ -61,10 +62,11 @@ public class DriverTaskTimeoutSentinelThreadTest {
QueryId queryId = new QueryId("test");
PlanFragmentId fragmentId = new PlanFragmentId(queryId, 0);
FragmentInstanceId instanceId = new FragmentInstanceId(fragmentId, "inst-0");
+ DriverTaskId driverTaskID = new DriverTaskId(instanceId, 0);
IndexedBlockingQueue<DriverTask> taskQueue =
new L1PriorityQueue<>(100, new DriverTask.TimeoutComparator(), new DriverTask());
IDriver mockDriver = Mockito.mock(IDriver.class);
- Mockito.when(mockDriver.getInfo()).thenReturn(instanceId);
+ Mockito.when(mockDriver.getDriverTaskId()).thenReturn(driverTaskID);
AbstractDriverThread executor =
new DriverTaskThread(
@@ -124,7 +126,8 @@ public class DriverTaskTimeoutSentinelThreadTest {
QueryId queryId = new QueryId("test");
PlanFragmentId fragmentId = new PlanFragmentId(queryId, 0);
FragmentInstanceId instanceId = new FragmentInstanceId(fragmentId, "inst-0");
- Mockito.when(mockDriver.getInfo()).thenReturn(instanceId);
+ DriverTaskId driverTaskID = new DriverTaskId(instanceId, 0);
+ Mockito.when(mockDriver.getDriverTaskId()).thenReturn(driverTaskID);
Mockito.when(mockDriver.processFor(Mockito.any()))
.thenReturn(Futures.immediateCancelledFuture());
@@ -135,7 +138,7 @@ public class DriverTaskTimeoutSentinelThreadTest {
executor.execute(testTask);
Mockito.verify(mockDriver, Mockito.times(1)).processFor(Mockito.any());
Assert.assertEquals(
- FragmentInstanceAbortedException.BY_ALREADY_BEING_CANCELLED, testTask.getAbortCause());
+ DriverTaskAbortedException.BY_ALREADY_BEING_CANCELLED, testTask.getAbortCause());
Mockito.verify(mockScheduler, Mockito.times(1)).toAborted(Mockito.any());
Mockito.verify(mockScheduler, Mockito.never()).runningToReady(Mockito.any(), Mockito.any());
Mockito.verify(mockScheduler, Mockito.never()).runningToBlocked(Mockito.any(), Mockito.any());
@@ -164,7 +167,8 @@ public class DriverTaskTimeoutSentinelThreadTest {
QueryId queryId = new QueryId("test");
PlanFragmentId fragmentId = new PlanFragmentId(queryId, 0);
FragmentInstanceId instanceId = new FragmentInstanceId(fragmentId, "inst-0");
- Mockito.when(mockDriver.getInfo()).thenReturn(instanceId);
+ DriverTaskId driverTaskID = new DriverTaskId(instanceId, 0);
+ Mockito.when(mockDriver.getDriverTaskId()).thenReturn(driverTaskID);
Mockito.when(mockDriver.processFor(Mockito.any()))
.thenAnswer(ans -> Futures.immediateVoidFuture());
Mockito.when(mockDriver.isFinished()).thenReturn(true);
@@ -214,7 +218,8 @@ public class DriverTaskTimeoutSentinelThreadTest {
QueryId queryId = new QueryId("test");
PlanFragmentId fragmentId = new PlanFragmentId(queryId, 0);
FragmentInstanceId instanceId = new FragmentInstanceId(fragmentId, "inst-0");
- Mockito.when(mockDriver.getInfo()).thenReturn(instanceId);
+ DriverTaskId driverTaskID = new DriverTaskId(instanceId, 0);
+ Mockito.when(mockDriver.getDriverTaskId()).thenReturn(driverTaskID);
Mockito.when(mockDriver.processFor(Mockito.any())).thenAnswer(ans -> mockFuture);
Mockito.when(mockDriver.isFinished()).thenReturn(false);
AbstractDriverThread executor =
@@ -264,7 +269,8 @@ public class DriverTaskTimeoutSentinelThreadTest {
QueryId queryId = new QueryId("test");
PlanFragmentId fragmentId = new PlanFragmentId(queryId, 0);
FragmentInstanceId instanceId = new FragmentInstanceId(fragmentId, "inst-0");
- Mockito.when(mockDriver.getInfo()).thenReturn(instanceId);
+ DriverTaskId driverTaskID = new DriverTaskId(instanceId, 0);
+ Mockito.when(mockDriver.getDriverTaskId()).thenReturn(driverTaskID);
Mockito.when(mockDriver.processFor(Mockito.any())).thenAnswer(ans -> mockFuture);
Mockito.when(mockDriver.isFinished()).thenReturn(false);
AbstractDriverThread executor =
@@ -300,7 +306,8 @@ public class DriverTaskTimeoutSentinelThreadTest {
QueryId queryId = new QueryId("test");
PlanFragmentId fragmentId = new PlanFragmentId(queryId, 0);
FragmentInstanceId instanceId = new FragmentInstanceId(fragmentId, "inst-0");
- Mockito.when(mockDriver.getInfo()).thenReturn(instanceId);
+ DriverTaskId driverTaskID = new DriverTaskId(instanceId, 0);
+ Mockito.when(mockDriver.getDriverTaskId()).thenReturn(driverTaskID);
AbstractDriverThread executor =
new DriverTaskThread(
"0", new ThreadGroup("timeout-test"), taskQueue, mockScheduler, producer);
@@ -315,7 +322,7 @@ public class DriverTaskTimeoutSentinelThreadTest {
executor.run(); // Here we use run() instead of start() to execute the task in the same thread
Mockito.verify(mockDriver, Mockito.times(1)).processFor(Mockito.any());
Assert.assertEquals(
- FragmentInstanceAbortedException.BY_INTERNAL_ERROR_SCHEDULED, testTask.getAbortCause());
+ DriverTaskAbortedException.BY_INTERNAL_ERROR_SCHEDULED, testTask.getAbortCause());
Assert.assertEquals(0, taskQueue.size());
Mockito.verify(mockScheduler, Mockito.times(1)).toAborted(Mockito.any());
Mockito.verify(mockScheduler, Mockito.never()).runningToReady(Mockito.any(), Mockito.any());
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/queue/L1PriorityQueueTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/queue/L1PriorityQueueTest.java
index a1db5ba8fb..5d36688454 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/queue/L1PriorityQueueTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/queue/L1PriorityQueueTest.java
@@ -58,7 +58,7 @@ public class L1PriorityQueueTest {
Thread.sleep(100);
Assert.assertEquals(Thread.State.TERMINATED, t1.getState());
Assert.assertEquals(1, res.size());
- Assert.assertEquals(e2.getId().toString(), res.get(0).getId().toString());
+ Assert.assertEquals(e2.getDriverTaskId().toString(), res.get(0).getDriverTaskId().toString());
}
@Test
@@ -102,9 +102,9 @@ public class L1PriorityQueueTest {
QueueElement e2 = new QueueElement(new QueueElement.QueueElementID(2), 5);
queue.push(e2);
Assert.assertEquals(2, queue.size());
- Assert.assertEquals(e2.getId().toString(), queue.poll().getId().toString());
+ Assert.assertEquals(e2.getDriverTaskId().toString(), queue.poll().getDriverTaskId().toString());
Assert.assertEquals(1, queue.size());
- Assert.assertEquals(e1.getId().toString(), queue.poll().getId().toString());
+ Assert.assertEquals(e1.getDriverTaskId().toString(), queue.poll().getDriverTaskId().toString());
Assert.assertEquals(0, queue.size());
}
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/queue/L2PriorityQueueTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/queue/L2PriorityQueueTest.java
index cb94a0703d..2c762e7736 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/queue/L2PriorityQueueTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/queue/L2PriorityQueueTest.java
@@ -63,7 +63,7 @@ public class L2PriorityQueueTest {
.atMost(1, TimeUnit.MINUTES)
.untilAsserted(() -> Assert.assertEquals(Thread.State.TERMINATED, t1.getState()));
Assert.assertEquals(1, res.size());
- Assert.assertEquals(e2.getId().toString(), res.get(0).getId().toString());
+ Assert.assertEquals(e2.getDriverTaskId().toString(), res.get(0).getDriverTaskId().toString());
}
@Test
@@ -103,7 +103,7 @@ public class L2PriorityQueueTest {
return res;
}
return String.CASE_INSENSITIVE_ORDER.compare(
- o1.getId().toString(), o2.getId().toString());
+ o1.getDriverTaskId().toString(), o2.getDriverTaskId().toString());
},
new QueueElement(new QueueElement.QueueElementID(0), 0));
QueueElement e1 = new QueueElement(new QueueElement.QueueElementID(1), 10);
@@ -112,14 +112,14 @@ public class L2PriorityQueueTest {
QueueElement e2 = new QueueElement(new QueueElement.QueueElementID(2), 5);
queue.push(e2);
Assert.assertEquals(2, queue.size());
- Assert.assertEquals(e2.getId().toString(), queue.poll().getId().toString());
+ Assert.assertEquals(e2.getDriverTaskId().toString(), queue.poll().getDriverTaskId().toString());
Assert.assertEquals(1, queue.size());
// L1: 5 -> 20 L2: 10
QueueElement e3 = new QueueElement(new QueueElement.QueueElementID(3), 10);
queue.push(e3);
- Assert.assertEquals(e1.getId().toString(), queue.poll().getId().toString());
+ Assert.assertEquals(e1.getDriverTaskId().toString(), queue.poll().getDriverTaskId().toString());
Assert.assertEquals(1, queue.size());
- Assert.assertEquals(e3.getId().toString(), queue.poll().getId().toString());
+ Assert.assertEquals(e3.getDriverTaskId().toString(), queue.poll().getDriverTaskId().toString());
Assert.assertEquals(0, queue.size());
}
@@ -161,7 +161,7 @@ public class L2PriorityQueueTest {
return res;
}
return String.CASE_INSENSITIVE_ORDER.compare(
- o1.getId().toString(), o2.getId().toString());
+ o1.getDriverTaskId().toString(), o2.getDriverTaskId().toString());
},
new QueueElement(new QueueElement.QueueElementID(0), 0));
QueueElement e1 = new QueueElement(new QueueElement.QueueElementID(1), 5);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/queue/QueueElement.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/queue/QueueElement.java
index 0047b5b7a7..215835f332 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/queue/QueueElement.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/schedule/queue/QueueElement.java
@@ -32,7 +32,7 @@ public class QueueElement implements IDIndexedAccessible {
}
@Override
- public ID getId() {
+ public ID getDriverTaskId() {
return id;
}
@@ -48,7 +48,7 @@ public class QueueElement implements IDIndexedAccessible {
@Override
public boolean equals(Object o) {
- return o instanceof QueueElement && ((QueueElement) o).getId().equals(this.id);
+ return o instanceof QueueElement && ((QueueElement) o).getDriverTaskId().equals(this.id);
}
public static class QueueElementID implements ID {