You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2022/11/10 12:17:20 UTC
[iotdb] branch master updated: Remove old storage engine step two (#7959)
This is an automated email from the ASF dual-hosted git repository.
qiaojialin 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 3614ca7899 Remove old storage engine step two (#7959)
3614ca7899 is described below
commit 3614ca78993035f81b0eb42b7e909d6ddd80deb2
Author: Haonan <hh...@outlook.com>
AuthorDate: Thu Nov 10 20:17:14 2022 +0800
Remove old storage engine step two (#7959)
---
.../schemaregion/rocksdb/RSchemaRegion.java | 131 --
.../metadata/tagSchemaRegion/TagSchemaRegion.java | 85 --
.../selectinto/InsertTabletPlanGenerator.java | 252 ----
.../selectinto/InsertTabletPlansIterator.java | 190 ---
.../iotdb/db/engine/storagegroup/DataRegion.java | 14 -
.../db/engine/trigger/executor/TriggerEngine.java | 64 -
.../trigger/sink/local/LocalIoTDBHandler.java | 12 +-
.../iotdb/db/metadata/LocalSchemaProcessor.java | 33 -
.../apache/iotdb/db/metadata/idtable/IDTable.java | 3 +-
.../db/metadata/idtable/IDTableHashmapImpl.java | 134 --
.../db/metadata/schemaregion/ISchemaRegion.java | 3 -
.../schemaregion/SchemaRegionMemoryImpl.java | 211 ---
.../schemaregion/SchemaRegionSchemaFileImpl.java | 230 ---
.../metadata/schemaregion/SchemaRegionUtils.java | 48 -
.../planner/plan/node/write/InsertTabletNode.java | 2 +-
.../iotdb/db/protocol/influxdb/dto/IoTDBPoint.java | 14 -
.../db/protocol/influxdb/meta/TagInfoRecords.java | 24 -
.../db/qp/logical/crud/DeleteDataOperator.java | 15 +-
.../iotdb/db/qp/logical/crud/InsertOperator.java | 31 +-
.../db/qp/logical/sys/DeletePartitionOperator.java | 3 +-
.../apache/iotdb/db/qp/physical/PhysicalPlan.java | 24 -
.../db/qp/physical/crud/DeletePartitionPlan.java | 53 -
.../iotdb/db/qp/physical/crud/DeletePlan.java | 221 ---
.../qp/physical/crud/InsertMultiTabletsPlan.java | 453 ------
.../iotdb/db/qp/physical/crud/InsertPlan.java | 239 ----
.../iotdb/db/qp/physical/crud/InsertRowPlan.java | 812 -----------
.../physical/crud/InsertRowsOfOneDevicePlan.java | 296 ----
.../iotdb/db/qp/physical/crud/InsertRowsPlan.java | 302 ----
.../db/qp/physical/crud/InsertTabletPlan.java | 895 ------------
.../service/thrift/impl/ClientRPCServiceImpl.java | 10 +-
.../db/service/thrift/impl/TSServiceImpl.java | 548 +-------
.../java/org/apache/iotdb/db/utils/MemUtils.java | 54 -
.../apache/iotdb/db/wal/buffer/WALEntryType.java | 4 -
.../apache/iotdb/db/wal/buffer/WALInfoEntry.java | 2 +-
.../db/wal/recover/file/TsFilePlanRedoer.java | 28 -
.../apache/iotdb/db/metadata/SchemaBasicTest.java | 298 ----
.../org/apache/iotdb/db/metadata/TemplateTest.java | 14 -
.../db/metadata/idtable/IDTableLogFileTest.java | 239 ++--
.../iotdb/db/metadata/idtable/IDTableTest.java | 1479 ++++++++++----------
.../java/org/apache/iotdb/db/qp/PlannerTest.java | 69 +-
.../db/utils/writelog/LogWriterReaderTest.java | 50 +-
.../db/utils/writelog/MultiFileLogReaderTest.java | 15 +-
.../db/wal/recover/file/TsFilePlanRedoerTest.java | 4 +-
43 files changed, 944 insertions(+), 6664 deletions(-)
diff --git a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java
index e7a41ef35b..95bf1f77f4 100644
--- a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java
+++ b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java
@@ -35,7 +35,6 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.exception.metadata.AcquireLockTimeoutException;
import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
-import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
import org.apache.iotdb.db.exception.metadata.PathNotExistException;
@@ -67,15 +66,11 @@ import org.apache.iotdb.db.metadata.schemaregion.rocksdb.mnode.RMeasurementMNode
import org.apache.iotdb.db.metadata.template.Template;
import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
import org.apache.iotdb.db.mpp.common.schematree.DeviceSchemaInfo;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
import org.apache.iotdb.db.query.context.QueryContext;
import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
-import org.apache.iotdb.db.utils.EncodingInferenceUtils;
import org.apache.iotdb.db.utils.SchemaUtils;
import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -102,7 +97,6 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
-import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
@@ -1837,131 +1831,6 @@ public class RSchemaRegion implements ISchemaRegion {
}
}
- @Override
- public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
- throws MetadataException, IOException {
- // devicePath is a logical path which is parent of measurement, whether in template or not
- PartialPath devicePath = plan.getDevicePath();
- String[] measurementList = plan.getMeasurements();
- IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
-
- IMNode deviceMNode = getDeviceNodeWithAutoCreate(devicePath, plan.isAligned());
-
- if (deviceMNode == null) {
- throw new MetadataException(
- String.format("Failed to create deviceMNode,device path:[%s]", plan.getDevicePath()));
- }
- // check insert non-aligned InsertPlan for aligned timeseries
- if (deviceMNode.isEntity()) {
- if (plan.isAligned() && !deviceMNode.getAsEntityMNode().isAligned()) {
- throw new AlignedTimeseriesException(
- "timeseries under this device are not aligned, " + "please use non-aligned interface",
- devicePath.getFullPath());
- }
-
- if (!plan.isAligned() && deviceMNode.getAsEntityMNode().isAligned()) {
- throw new AlignedTimeseriesException(
- "timeseries under this device are aligned, " + "please use aligned interface",
- devicePath.getFullPath());
- }
- }
-
- // get node for each measurement
- Map<Integer, IMeasurementMNode> nodeMap = new HashMap<>();
- Map<Integer, PartialPath> missingNodeIndex = new HashMap<>();
- for (int i = 0; i < measurementList.length; i++) {
- PartialPath path = new PartialPath(devicePath.getFullPath(), measurementList[i]);
- IMeasurementMNode node = getMeasurementMNode(path);
- if (node == null) {
- if (!config.isAutoCreateSchemaEnabled()) {
- throw new PathNotExistException(path.getFullPath());
- }
- missingNodeIndex.put(i, path);
- } else {
- nodeMap.put(i, node);
- }
- }
-
- // create missing nodes
- if (!missingNodeIndex.isEmpty()) {
- if (!(plan instanceof InsertRowPlan) && !(plan instanceof InsertTabletPlan)) {
- throw new MetadataException(
- String.format(
- "Only support insertRow and insertTablet, plan is [%s]", plan.getOperatorType()));
- }
-
- if (plan.isAligned()) {
- List<String> measurements = new ArrayList<>();
- List<TSDataType> dataTypes = new ArrayList<>();
- List<TSEncoding> encodings = new ArrayList<>();
- for (Integer index : missingNodeIndex.keySet()) {
- measurements.add(measurementList[index]);
- TSDataType type = plan.getDataTypes()[index];
- dataTypes.add(type);
- encodings.add(EncodingInferenceUtils.getDefaultEncoding(type));
- }
- createAlignedTimeSeries(devicePath, measurements, dataTypes, encodings);
- } else {
- for (Map.Entry<Integer, PartialPath> entry : missingNodeIndex.entrySet()) {
- IMeasurementSchema schema =
- new MeasurementSchema(
- entry.getValue().getMeasurement(), plan.getDataTypes()[entry.getKey()]);
- createTimeseries(entry.getValue(), schema, null, null, null);
- }
- }
-
- // get the latest node
- for (Entry<Integer, PartialPath> entry : missingNodeIndex.entrySet()) {
- nodeMap.put(entry.getKey(), getMeasurementMNode(entry.getValue()));
- }
- }
-
- // check datatype
- for (int i = 0; i < measurementList.length; i++) {
- try {
- // check type is match
- if (plan instanceof InsertRowPlan || plan instanceof InsertTabletPlan) {
- try {
- SchemaRegionUtils.checkDataTypeMatch(plan, i, nodeMap.get(i).getSchema().getType());
- } catch (DataTypeMismatchException mismatchException) {
- logger.warn(mismatchException.getMessage());
- if (!config.isEnablePartialInsert()) {
- throw mismatchException;
- } else {
- // mark failed measurement
- plan.markFailedMeasurementInsertion(i, mismatchException);
- continue;
- }
- }
- measurementMNodes[i] = nodeMap.get(i);
- // set measurementName instead of alias
- measurementList[i] = nodeMap.get(i).getName();
- }
- } catch (MetadataException e) {
- if (config.isClusterMode()) {
- logger.debug(
- "meet error when check {}.{}, message: {}",
- devicePath,
- measurementList[i],
- e.getMessage());
- } else {
- logger.warn(
- "meet error when check {}.{}, message: {}",
- devicePath,
- measurementList[i],
- e.getMessage());
- }
- if (config.isEnablePartialInsert()) {
- // mark failed measurement
- plan.markFailedMeasurementInsertion(i, e);
- } else {
- throw e;
- }
- }
- }
- return deviceMNode;
- }
-
@Override
public DeviceSchemaInfo getDeviceSchemaInfoWithAutoCreate(
PartialPath devicePath,
diff --git a/schema-engine-tag/src/main/java/org/apache/iotdb/db/metadata/tagSchemaRegion/TagSchemaRegion.java b/schema-engine-tag/src/main/java/org/apache/iotdb/db/metadata/tagSchemaRegion/TagSchemaRegion.java
index 38cc03c953..174f989e86 100644
--- a/schema-engine-tag/src/main/java/org/apache/iotdb/db/metadata/tagSchemaRegion/TagSchemaRegion.java
+++ b/schema-engine-tag/src/main/java/org/apache/iotdb/db/metadata/tagSchemaRegion/TagSchemaRegion.java
@@ -30,7 +30,6 @@ import org.apache.iotdb.commons.utils.TestOnly;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
-import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
import org.apache.iotdb.db.exception.metadata.PathNotExistException;
import org.apache.iotdb.db.exception.metadata.SchemaDirCreationFailureException;
@@ -38,7 +37,6 @@ import org.apache.iotdb.db.metadata.LocalSchemaProcessor;
import org.apache.iotdb.db.metadata.idtable.entry.DeviceEntry;
import org.apache.iotdb.db.metadata.idtable.entry.DiskSchemaEntry;
import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
-import org.apache.iotdb.db.metadata.idtable.entry.InsertMeasurementMNode;
import org.apache.iotdb.db.metadata.idtable.entry.SHA256DeviceID;
import org.apache.iotdb.db.metadata.idtable.entry.SchemaEntry;
import org.apache.iotdb.db.metadata.mnode.EntityMNode;
@@ -65,7 +63,6 @@ import org.apache.iotdb.db.metadata.tagSchemaRegion.utils.ShowTimeSeriesResultUt
import org.apache.iotdb.db.metadata.template.Template;
import org.apache.iotdb.db.mpp.common.schematree.DeviceSchemaInfo;
import org.apache.iotdb.db.mpp.common.schematree.MeasurementSchemaInfo;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
@@ -87,7 +84,6 @@ import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.LinkedList;
@@ -739,44 +735,6 @@ public class TagSchemaRegion implements ISchemaRegion {
throw new UnsupportedOperationException("renameTagOrAttributeKey");
}
- @Override
- public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
- throws MetadataException, IOException {
- PartialPath devicePath = plan.getDevicePath();
- devicePath =
- new PartialPath(
- PathTagConverterUtils.pathToTagsSortPath(
- storageGroupFullPath, devicePath.getFullPath()));
- plan.setDevicePath(devicePath);
- String[] measurementList = plan.getMeasurements();
- IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
- checkAlignedAndAutoCreateSeries(plan);
- IMNode deviceMNode = getDeviceNode(devicePath);
- IMeasurementMNode measurementMNode;
- DeviceEntry deviceEntry = idTableWithDeviceIDList.getDeviceEntry(devicePath.getFullPath());
- Map<String, SchemaEntry> schemaMap = deviceEntry.getMeasurementMap();
- for (int i = 0; i < measurementList.length; i++) {
- SchemaEntry schemaEntry = schemaMap.get(measurementList[i]);
- measurementMNode = new InsertMeasurementMNode(measurementList[i], schemaEntry, null);
- // check type is match
- try {
- SchemaRegionUtils.checkDataTypeMatch(plan, i, schemaEntry.getTSDataType());
- } catch (DataTypeMismatchException mismatchException) {
- if (!config.isEnablePartialInsert()) {
- throw mismatchException;
- } else {
- // mark failed measurement
- plan.markFailedMeasurementInsertion(i, mismatchException);
- continue;
- }
- }
- measurementMNodes[i] = measurementMNode;
- }
- plan.setDeviceID(deviceEntry.getDeviceID());
- plan.setDevicePath(new PartialPath(deviceEntry.getDeviceID().toStringID(), false));
- return deviceMNode;
- }
-
@Override
public DeviceSchemaInfo getDeviceSchemaInfoWithAutoCreate(
PartialPath devicePath,
@@ -833,36 +791,6 @@ public class TagSchemaRegion implements ISchemaRegion {
return deviceEntry.getSchemaEntry(measurementName);
}
- private void checkAlignedAndAutoCreateSeries(InsertPlan plan) throws MetadataException {
- String[] measurementList = plan.getMeasurements();
- try {
- if (plan.isAligned()) {
- internalAlignedCreateTimeseries(
- plan.getDevicePath(),
- Arrays.asList(measurementList),
- Arrays.asList(plan.getDataTypes()));
- } else {
- internalCreateTimeseries(
- plan.getDevicePath().concatNode(measurementList[0]), plan.getDataTypes()[0]);
- }
- } catch (MetadataException e) {
- if (!(e instanceof PathAlreadyExistException)) {
- throw e;
- }
- }
- }
-
- /** create timeseries ignoring PathAlreadyExistException */
- private void internalCreateTimeseries(PartialPath path, TSDataType dataType)
- throws MetadataException {
- createTimeseries(
- path,
- dataType,
- getDefaultEncoding(dataType),
- TSFileDescriptor.getInstance().getConfig().getCompressor(),
- Collections.emptyMap());
- }
-
/** create timeseries ignoring PathAlreadyExistException */
private void internalCreateTimeseries(
PartialPath path, TSDataType dataType, TSEncoding encoding, CompressionType compressor)
@@ -876,19 +804,6 @@ public class TagSchemaRegion implements ISchemaRegion {
createTimeseries(path, dataType, encoding, compressor, Collections.emptyMap());
}
- /** create aligned timeseries ignoring PathAlreadyExistException */
- private void internalAlignedCreateTimeseries(
- PartialPath prefixPath, List<String> measurements, List<TSDataType> dataTypes)
- throws MetadataException {
- List<TSEncoding> encodings = new ArrayList<>();
- List<CompressionType> compressors = new ArrayList<>();
- for (TSDataType dataType : dataTypes) {
- encodings.add(getDefaultEncoding(dataType));
- compressors.add(TSFileDescriptor.getInstance().getConfig().getCompressor());
- }
- createAlignedTimeSeries(prefixPath, measurements, dataTypes, encodings, compressors);
- }
-
/** create aligned timeseries ignoring PathAlreadyExistException */
private void internalAlignedCreateTimeseries(
PartialPath prefixPath,
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/selectinto/InsertTabletPlanGenerator.java b/server/src/main/java/org/apache/iotdb/db/engine/selectinto/InsertTabletPlanGenerator.java
deleted file mode 100644
index affa549ddf..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/selectinto/InsertTabletPlanGenerator.java
+++ /dev/null
@@ -1,252 +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.engine.selectinto;
-
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
-import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.read.common.Field;
-import org.apache.iotdb.tsfile.read.common.RowRecord;
-import org.apache.iotdb.tsfile.utils.Binary;
-import org.apache.iotdb.tsfile.utils.BitMap;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-/** internallyConstructNewPlan -> collectRowRecord * N -> generateInsertTabletPlan */
-public class InsertTabletPlanGenerator {
-
- private final String targetDevice;
- // the index of column in InsertTabletPlan -> the index of output column of query data set
- private final List<Integer> queryDataSetIndexes;
- // the index of column in InsertTabletPlan -> the measurement id of the column
- private final List<String> targetMeasurementIds;
-
- private final int tabletRowLimit;
- private final boolean isIntoPathsAligned;
-
- // the following fields are used to construct plan
- private int rowCount;
- private long[] times;
- private Object[] columns;
- private BitMap[] bitMaps;
- private TSDataType[] dataTypes;
-
- private int numberOfInitializedColumns;
-
- public InsertTabletPlanGenerator(
- String targetDevice, int tabletRowLimit, boolean isIntoPathsAligned) {
- this.targetDevice = targetDevice;
- queryDataSetIndexes = new ArrayList<>();
- targetMeasurementIds = new ArrayList<>();
-
- this.tabletRowLimit = tabletRowLimit;
-
- this.isIntoPathsAligned = isIntoPathsAligned;
- }
-
- public void collectTargetPathInformation(String targetMeasurementId, int queryDataSetIndex) {
- targetMeasurementIds.add(targetMeasurementId);
- queryDataSetIndexes.add(queryDataSetIndex);
- }
-
- public void internallyConstructNewPlan() {
- rowCount = 0;
- times = new long[tabletRowLimit];
- columns = new Object[targetMeasurementIds.size()];
- bitMaps = new BitMap[targetMeasurementIds.size()];
- for (int i = 0; i < bitMaps.length; ++i) {
- bitMaps[i] = new BitMap(tabletRowLimit);
- bitMaps[i].markAll();
- }
- dataTypes = new TSDataType[targetMeasurementIds.size()];
-
- numberOfInitializedColumns = 0;
- }
-
- public void collectRowRecord(RowRecord rowRecord) {
- if (numberOfInitializedColumns != columns.length) {
- List<Integer> initializedDataTypeIndexes = trySetDataTypes(rowRecord);
- tryInitColumns(initializedDataTypeIndexes);
- numberOfInitializedColumns += initializedDataTypeIndexes.size();
- }
-
- times[rowCount] = rowRecord.getTimestamp();
-
- for (int i = 0; i < columns.length; ++i) {
- Field field = rowRecord.getFields().get(queryDataSetIndexes.get(i));
-
- // if the field is NULL
- if (field == null || field.getDataType() == null) {
- // bit in bitMaps are marked as 1 (NULL) by default
- continue;
- }
-
- bitMaps[i].unmark(rowCount);
- switch (field.getDataType()) {
- case INT32:
- ((int[]) columns[i])[rowCount] = field.getIntV();
- break;
- case INT64:
- ((long[]) columns[i])[rowCount] = field.getLongV();
- break;
- case FLOAT:
- ((float[]) columns[i])[rowCount] = field.getFloatV();
- break;
- case DOUBLE:
- ((double[]) columns[i])[rowCount] = field.getDoubleV();
- break;
- case BOOLEAN:
- ((boolean[]) columns[i])[rowCount] = field.getBoolV();
- break;
- case TEXT:
- ((Binary[]) columns[i])[rowCount] = field.getBinaryV();
- break;
- default:
- throw new UnSupportedDataTypeException(
- String.format(
- "data type %s is not supported when convert data at client",
- field.getDataType()));
- }
- }
-
- ++rowCount;
- }
-
- private List<Integer> trySetDataTypes(RowRecord rowRecord) {
- List<Integer> initializedDataTypeIndexes = new ArrayList<>();
- List<Field> fields = rowRecord.getFields();
-
- for (int i = 0; i < dataTypes.length; ++i) {
- // if the data type is already set
- if (dataTypes[i] != null) {
- continue;
- }
-
- // get the field index of the row record
- int queryDataSetIndex = queryDataSetIndexes.get(i);
- // if the field is not null
- if (fields.get(queryDataSetIndex) != null
- && fields.get(queryDataSetIndex).getDataType() != null) {
- // set the data type to the field type
- dataTypes[i] = fields.get(queryDataSetIndex).getDataType();
- initializedDataTypeIndexes.add(i);
- }
- }
- return initializedDataTypeIndexes;
- }
-
- private void tryInitColumns(List<Integer> initializedDataTypeIndexes) {
- for (int i : initializedDataTypeIndexes) {
- switch (dataTypes[i]) {
- case BOOLEAN:
- columns[i] = new boolean[tabletRowLimit];
- break;
- case INT32:
- columns[i] = new int[tabletRowLimit];
- break;
- case INT64:
- columns[i] = new long[tabletRowLimit];
- break;
- case FLOAT:
- columns[i] = new float[tabletRowLimit];
- break;
- case DOUBLE:
- columns[i] = new double[tabletRowLimit];
- break;
- case TEXT:
- columns[i] = new Binary[tabletRowLimit];
- Arrays.fill((Binary[]) columns[i], Binary.EMPTY_VALUE);
- break;
- default:
- throw new UnSupportedDataTypeException(
- String.format("Data type %s is not supported.", dataTypes[i]));
- }
- }
- }
-
- public InsertTabletPlan generateInsertTabletPlan() throws IllegalPathException {
- List<String> nonEmptyColumnNames = new ArrayList<>();
-
- int countOfNonEmptyColumns = 0;
- for (int i = 0; i < columns.length; ++i) {
- if (columns[i] == null) {
- continue;
- }
-
- nonEmptyColumnNames.add(targetMeasurementIds.get(i));
- columns[countOfNonEmptyColumns] = columns[i];
- bitMaps[countOfNonEmptyColumns] = bitMaps[i];
- dataTypes[countOfNonEmptyColumns] = dataTypes[i];
-
- ++countOfNonEmptyColumns;
- }
-
- InsertTabletPlan insertTabletPlan =
- new InsertTabletPlan(new PartialPath(targetDevice), nonEmptyColumnNames);
- insertTabletPlan.setAligned(isIntoPathsAligned);
- insertTabletPlan.setRowCount(rowCount);
-
- if (countOfNonEmptyColumns != columns.length) {
- columns = Arrays.copyOf(columns, countOfNonEmptyColumns);
- bitMaps = Arrays.copyOf(bitMaps, countOfNonEmptyColumns);
- dataTypes = Arrays.copyOf(dataTypes, countOfNonEmptyColumns);
- }
-
- if (rowCount != tabletRowLimit) {
- times = Arrays.copyOf(times, rowCount);
- for (int i = 0; i < columns.length; ++i) {
- switch (dataTypes[i]) {
- case BOOLEAN:
- columns[i] = Arrays.copyOf((boolean[]) columns[i], rowCount);
- break;
- case INT32:
- columns[i] = Arrays.copyOf((int[]) columns[i], rowCount);
- break;
- case INT64:
- columns[i] = Arrays.copyOf((long[]) columns[i], rowCount);
- break;
- case FLOAT:
- columns[i] = Arrays.copyOf((float[]) columns[i], rowCount);
- break;
- case DOUBLE:
- columns[i] = Arrays.copyOf((double[]) columns[i], rowCount);
- break;
- case TEXT:
- columns[i] = Arrays.copyOf((Binary[]) columns[i], rowCount);
- break;
- default:
- throw new UnSupportedDataTypeException(
- String.format("Data type %s is not supported.", dataTypes[i]));
- }
- }
- }
-
- insertTabletPlan.setTimes(times);
- insertTabletPlan.setColumns(columns);
- insertTabletPlan.setBitMaps(bitMaps);
- insertTabletPlan.setDataTypes(dataTypes);
-
- return insertTabletPlan;
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/selectinto/InsertTabletPlansIterator.java b/server/src/main/java/org/apache/iotdb/db/engine/selectinto/InsertTabletPlansIterator.java
deleted file mode 100644
index da291e6ef4..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/selectinto/InsertTabletPlansIterator.java
+++ /dev/null
@@ -1,190 +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.engine.selectinto;
-
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.mpp.plan.expression.ResultColumn;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
-import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
-import org.apache.iotdb.tsfile.read.common.Path;
-import org.apache.iotdb.tsfile.read.common.RowRecord;
-import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-public class InsertTabletPlansIterator {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(InsertTabletPlansIterator.class);
-
- private static final Pattern LEVELED_PATH_NODE_PATTERN = Pattern.compile("\\$\\{\\w+}");
-
- private final QueryPlan queryPlan;
- private final QueryDataSet queryDataSet;
-
- private final PartialPath fromPath;
- private final List<PartialPath> intoPaths;
- private final boolean isIntoPathsAligned;
-
- private final int tabletRowLimit;
-
- private InsertTabletPlanGenerator[] insertTabletPlanGenerators;
-
- public InsertTabletPlansIterator(
- QueryPlan queryPlan,
- QueryDataSet queryDataSet,
- PartialPath fromPath,
- List<PartialPath> intoPaths,
- boolean isIntoPathsAligned)
- throws IllegalPathException {
- this.queryPlan = queryPlan;
- this.queryDataSet = queryDataSet;
- this.fromPath = fromPath;
- this.intoPaths = intoPaths;
- this.isIntoPathsAligned = isIntoPathsAligned;
-
- tabletRowLimit =
- IoTDBDescriptor.getInstance().getConfig().getSelectIntoInsertTabletPlanRowLimit();
-
- generateActualIntoPaths();
- constructInsertTabletPlanGenerators();
- }
-
- private void generateActualIntoPaths() throws IllegalPathException {
- for (int i = 0; i < intoPaths.size(); ++i) {
- intoPaths.set(i, generateActualIntoPath(i));
- }
- }
-
- private PartialPath generateActualIntoPath(int index) throws IllegalPathException {
- String[] nodes = fromPath.getNodes();
- StringBuffer sb = new StringBuffer();
- Matcher m = LEVELED_PATH_NODE_PATTERN.matcher(intoPaths.get(index).getFullPath());
- while (m.find()) {
- String param = m.group();
- String value = nodes[Integer.parseInt(param.substring(2, param.length() - 1).trim())];
- m.appendReplacement(sb, value == null ? "" : value);
- }
- m.appendTail(sb);
- return new PartialPath(sb.toString());
- }
-
- private void constructInsertTabletPlanGenerators() {
- final Map<String, Integer> sourcePathToQueryDataSetIndex = queryPlan.getPathToIndex();
- final List<String> sourcePaths = findSourcePaths();
-
- Map<String, InsertTabletPlanGenerator> deviceToPlanGeneratorMap = new HashMap<>();
- for (int i = 0, intoPathsSize = intoPaths.size(); i < intoPathsSize; i++) {
- String device = intoPaths.get(i).getDevice();
- if (!deviceToPlanGeneratorMap.containsKey(device)) {
- deviceToPlanGeneratorMap.put(
- device, new InsertTabletPlanGenerator(device, tabletRowLimit, isIntoPathsAligned));
- }
- deviceToPlanGeneratorMap
- .get(device)
- .collectTargetPathInformation(
- intoPaths.get(i).getMeasurement(),
- sourcePathToQueryDataSetIndex.get(sourcePaths.get(i)));
- }
-
- insertTabletPlanGenerators =
- deviceToPlanGeneratorMap.values().toArray(new InsertTabletPlanGenerator[0]);
- }
-
- private List<String> findSourcePaths() {
- // sourcePaths can be in queryPlanColumns or in queryDataSetPaths
- final List<ResultColumn> queryPlanColumns = queryPlan.getResultColumns();
- final List<Path> queryDataSetPaths = queryDataSet.getPaths();
-
- final Map<String, Integer> sourcePathToQueryDataSetIndex = queryPlan.getPathToIndex();
- final List<String> sourcePaths =
- new ArrayList<>(Math.max(queryPlanColumns.size(), queryDataSetPaths.size()));
-
- if (queryPlanColumns.size() == intoPaths.size()) {
- for (ResultColumn resultColumn : queryPlanColumns) {
- String path = resultColumn.getResultColumnName();
- if (!sourcePathToQueryDataSetIndex.containsKey(path)) {
- sourcePaths.clear();
- break;
- }
- sourcePaths.add(path);
- }
- }
-
- if (sourcePaths.isEmpty() && queryDataSetPaths.size() == intoPaths.size()) {
- for (Path path : queryDataSetPaths) {
- if (!sourcePathToQueryDataSetIndex.containsKey(path.getFullPath())) {
- sourcePaths.clear();
- break;
- }
- sourcePaths.add(path.getFullPath());
- }
- }
-
- if (sourcePaths.isEmpty()) {
- LOGGER.warn("select into: sourcePaths.isEmpty()");
- }
-
- return sourcePaths;
- }
-
- public boolean hasNext() throws IOException {
- return queryDataSet.hasNext();
- }
-
- public List<InsertTabletPlan> next() throws IOException, IllegalPathException {
- for (InsertTabletPlanGenerator insertTabletPlanGenerator : insertTabletPlanGenerators) {
- insertTabletPlanGenerator.internallyConstructNewPlan();
- }
-
- collectRowRecordIntoInsertTabletPlanGenerators();
-
- List<InsertTabletPlan> insertTabletPlans = new ArrayList<>();
- for (InsertTabletPlanGenerator insertTabletPlanGenerator : insertTabletPlanGenerators) {
- // all values can be null in a batch of the query dataset
- InsertTabletPlan insertTabletPlan = insertTabletPlanGenerator.generateInsertTabletPlan();
- if (insertTabletPlan.getColumns().length != 0) {
- insertTabletPlans.add(insertTabletPlan);
- }
- }
- return insertTabletPlans;
- }
-
- private void collectRowRecordIntoInsertTabletPlanGenerators() throws IOException {
- int count = 0;
- while (queryDataSet.hasNext() && count < tabletRowLimit) {
- RowRecord rowRecord = queryDataSet.next();
- for (InsertTabletPlanGenerator insertTabletPlanGenerator : insertTabletPlanGenerators) {
- insertTabletPlanGenerator.collectRowRecord(rowRecord);
- }
- ++count;
- }
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java
index 75b26d34ff..69b5e9db6f 100755
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java
@@ -79,7 +79,6 @@ import org.apache.iotdb.db.query.context.QueryContext;
import org.apache.iotdb.db.query.control.FileReaderManager;
import org.apache.iotdb.db.query.control.QueryFileManager;
import org.apache.iotdb.db.rescon.TsFileResourceManager;
-import org.apache.iotdb.db.service.IoTDB;
import org.apache.iotdb.db.service.SettleService;
import org.apache.iotdb.db.sync.SyncService;
import org.apache.iotdb.db.sync.sender.manager.ISyncManager;
@@ -2062,19 +2061,6 @@ public class DataRegion {
}
}
- private void tryToDeleteLastCache(
- PartialPath deviceId, PartialPath originalPath, long startTime, long endTime)
- throws WriteProcessException {
- if (!IoTDBDescriptor.getInstance().getConfig().isLastCacheEnabled()) {
- return;
- }
- try {
- IoTDB.schemaProcessor.deleteLastCacheByDevice(deviceId, originalPath, startTime, endTime);
- } catch (MetadataException e) {
- throw new WriteProcessException(e);
- }
- }
-
private boolean unsequenceFlushCallback(
TsFileProcessor processor, Map<String, Long> updateMap, long systemFlushTime) {
TimePartitionManager.getInstance()
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/trigger/executor/TriggerEngine.java b/server/src/main/java/org/apache/iotdb/db/engine/trigger/executor/TriggerEngine.java
index 07605f6d07..96ccf5598a 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/trigger/executor/TriggerEngine.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/trigger/executor/TriggerEngine.java
@@ -21,83 +21,19 @@ package org.apache.iotdb.db.engine.trigger.executor;
import org.apache.iotdb.db.engine.trigger.service.TriggerRegistrationInformation;
import org.apache.iotdb.db.engine.trigger.service.TriggerRegistrationService;
-import org.apache.iotdb.db.exception.TriggerExecutionException;
import org.apache.iotdb.db.exception.TriggerManagementException;
import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
import org.apache.iotdb.db.qp.physical.sys.DropTriggerPlan;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.util.Arrays;
import java.util.List;
public class TriggerEngine {
private static final Logger LOGGER = LoggerFactory.getLogger(TriggerEngine.class);
- public static void fire(TriggerEvent event, InsertRowPlan insertRowPlan)
- throws TriggerExecutionException {
- if (TriggerRegistrationService.getInstance().executorSize() == 0) {
- return;
- }
-
- IMeasurementMNode[] mNodes = insertRowPlan.getMeasurementMNodes();
- int size = mNodes.length;
-
- long timestamp = insertRowPlan.getTime();
- Object[] values = insertRowPlan.getValues();
-
- for (int i = 0; i < size; ++i) {
- IMeasurementMNode mNode = mNodes[i];
- if (mNode == null) {
- continue;
- }
- for (TriggerExecutor executor : mNode.getUpperTriggerExecutorList()) {
- executor.fireIfActivated(
- event,
- timestamp,
- values[i],
- mNode.getSchema().getType(),
- insertRowPlan.getPaths().get(i));
- }
- }
- }
-
- public static void fire(TriggerEvent event, InsertTabletPlan insertTabletPlan, int firePosition)
- throws TriggerExecutionException {
- if (TriggerRegistrationService.getInstance().executorSize() == 0) {
- return;
- }
-
- IMeasurementMNode[] mNodes = insertTabletPlan.getMeasurementMNodes();
- int size = mNodes.length;
-
- long[] timestamps = insertTabletPlan.getTimes();
- Object[] columns = insertTabletPlan.getColumns();
- if (firePosition != 0) {
- timestamps = Arrays.copyOfRange(timestamps, firePosition, timestamps.length);
- columns = Arrays.copyOfRange(columns, firePosition, columns.length);
- }
-
- for (int i = 0; i < size; ++i) {
- IMeasurementMNode mNode = mNodes[i];
- if (mNode == null) {
- continue;
- }
- for (TriggerExecutor executor : mNode.getUpperTriggerExecutorList()) {
- executor.fireIfActivated(
- event,
- timestamps,
- columns[i],
- mNode.getSchema().getType(),
- insertTabletPlan.getPaths().get(i));
- }
- }
- }
-
public static void drop(IMeasurementMNode measurementMNode) {
TriggerExecutor executor = measurementMNode.getTriggerExecutor();
if (executor == null) {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/trigger/sink/local/LocalIoTDBHandler.java b/server/src/main/java/org/apache/iotdb/db/engine/trigger/sink/local/LocalIoTDBHandler.java
index a5567b5a7c..7d73860c40 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/trigger/sink/local/LocalIoTDBHandler.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/trigger/sink/local/LocalIoTDBHandler.java
@@ -30,7 +30,6 @@ import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.qp.executor.IPlanExecutor;
import org.apache.iotdb.db.qp.executor.PlanExecutor;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
import org.apache.iotdb.db.service.IoTDB;
import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -82,16 +81,7 @@ public class LocalIoTDBHandler implements Handler<LocalIoTDBConfiguration, Local
@Override
public void onEvent(LocalIoTDBEvent event)
- throws QueryProcessException, StorageEngineException, StorageGroupNotSetException {
- InsertRowPlan plan = new InsertRowPlan();
- plan.setNeedInferType(false);
- plan.setDevicePath(device);
- plan.setMeasurements(measurements);
- plan.setDataTypes(dataTypes);
- plan.setTime(event.getTimestamp());
- plan.setValues(event.getValues());
- executeNonQuery(plan);
- }
+ throws QueryProcessException, StorageEngineException, StorageGroupNotSetException {}
private void executeNonQuery(PhysicalPlan plan)
throws QueryProcessException, StorageGroupNotSetException, StorageEngineException {
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/LocalSchemaProcessor.java b/server/src/main/java/org/apache/iotdb/db/metadata/LocalSchemaProcessor.java
index 273c4a179a..100511c9e9 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/LocalSchemaProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/LocalSchemaProcessor.java
@@ -44,7 +44,6 @@ import org.apache.iotdb.db.metadata.template.Template;
import org.apache.iotdb.db.metadata.template.TemplateManager;
import org.apache.iotdb.db.qp.constant.SQLConstant;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
import org.apache.iotdb.db.qp.physical.sys.AppendTemplatePlan;
import org.apache.iotdb.db.qp.physical.sys.AutoCreateDeviceMNodePlan;
@@ -119,7 +118,6 @@ import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARA
* <li>Interfaces for alias and tag/attribute operations
* <li>Interfaces only for Cluster module usage
* <li>Interfaces for lastCache operations
- * <li>Interfaces and Implementation for InsertPlan process
* <li>Interfaces and Implementation for Template operations
* <li>Interfaces for Trigger
* <li>TestOnly Interfaces
@@ -1224,20 +1222,6 @@ public class LocalSchemaProcessor {
LastCacheManager.resetLastCache(node);
}
- /**
- * delete all the last cache value of any timeseries or aligned timeseries under the device
- *
- * <p>Invoking scenario (1) after upload tsfile
- *
- * @param deviceId path of device
- */
- public void deleteLastCacheByDevice(PartialPath deviceId) throws MetadataException {
- IMNode node = getDeviceNode(deviceId);
- if (node.isEntity()) {
- LastCacheManager.deleteLastCacheByDevice(node.getAsEntityMNode());
- }
- }
-
/**
* delete the last cache value of timeseries or subMeasurement of some aligned timeseries, which
* is under the device and matching the originalPath
@@ -1260,23 +1244,6 @@ public class LocalSchemaProcessor {
}
// endregion
- // region Interfaces and Implementation for InsertPlan process
- /** get schema for device. Attention!!! Only support insertPlan */
- @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
- public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
- throws MetadataException, IOException {
- ISchemaRegion schemaRegion;
- if (config.isAutoCreateSchemaEnabled()) {
- schemaRegion = getBelongedSchemaRegionWithAutoCreate(plan.getDevicePath());
- } else {
- schemaRegion = getBelongedSchemaRegion(plan.getDevicePath());
- }
-
- return schemaRegion.getSeriesSchemasAndReadLockDevice(plan);
- }
-
- // endregion
-
// region Interfaces and Implementation for Template operations
public void createSchemaTemplate(CreateTemplatePlan plan) throws MetadataException {
configManager.createSchemaTemplate(plan);
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTable.java b/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTable.java
index 8723e26375..a067cf98b8 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTable.java
@@ -35,7 +35,6 @@ import org.apache.iotdb.db.metadata.idtable.entry.TimeseriesID;
import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateAlignedTimeSeriesPlan;
import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateTimeSeriesPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
import org.apache.iotdb.tsfile.read.TimeValuePair;
import org.apache.iotdb.tsfile.utils.Pair;
import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
@@ -87,7 +86,7 @@ public interface IDTable {
* @return reusable device id
* @throws MetadataException if insert plan's aligned value is inconsistent with device
*/
- IDeviceID getSeriesSchemas(InsertPlan plan) throws MetadataException;
+ // IDeviceID getSeriesSchemas(InsertPlan plan) throws MetadataException;
/**
* register trigger to the timeseries
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableHashmapImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableHashmapImpl.java
index 5880cb7c28..31a39a45a2 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableHashmapImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableHashmapImpl.java
@@ -24,20 +24,15 @@ import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.commons.utils.TestOnly;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
import org.apache.iotdb.db.metadata.idtable.entry.DeviceEntry;
import org.apache.iotdb.db.metadata.idtable.entry.DeviceIDFactory;
import org.apache.iotdb.db.metadata.idtable.entry.DiskSchemaEntry;
import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
-import org.apache.iotdb.db.metadata.idtable.entry.InsertMeasurementMNode;
import org.apache.iotdb.db.metadata.idtable.entry.SchemaEntry;
import org.apache.iotdb.db.metadata.idtable.entry.TimeseriesID;
import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateAlignedTimeSeriesPlan;
import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateTimeSeriesPlan;
-import org.apache.iotdb.db.metadata.schemaregion.SchemaRegionUtils;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
-import org.apache.iotdb.db.service.IoTDB;
import org.apache.iotdb.tsfile.read.TimeValuePair;
import org.apache.iotdb.tsfile.utils.Pair;
import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
@@ -176,73 +171,6 @@ public class IDTableHashmapImpl implements IDTable {
return new Pair<>(deletedNum, failedNames);
}
- /**
- * check inserting timeseries existence and fill their measurement mnode
- *
- * @param plan insert plan
- * @return reusable device id
- * @throws MetadataException if insert plan's aligned value is inconsistent with device
- */
- @Override
- public synchronized IDeviceID getSeriesSchemas(InsertPlan plan) throws MetadataException {
- PartialPath devicePath = plan.getDevicePath();
- String[] measurementList = plan.getMeasurements();
- IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
-
- // 1. get device entry and check align
- DeviceEntry deviceEntry =
- getDeviceEntryWithAlignedCheck(devicePath.toString(), plan.isAligned());
-
- // 2. get schema of each measurement
- for (int i = 0; i < measurementList.length; i++) {
- try {
- // get MeasurementMNode, auto create if absent
- try {
- IMeasurementMNode measurementMNode =
- getOrCreateMeasurementIfNotExist(deviceEntry, plan, i);
-
- SchemaRegionUtils.checkDataTypeMatch(plan, i, measurementMNode.getSchema().getType());
- measurementMNodes[i] = measurementMNode;
- } catch (DataTypeMismatchException mismatchException) {
- logger.warn(mismatchException.getMessage());
- if (!config.isEnablePartialInsert()) {
- throw mismatchException;
- } else {
- // mark failed measurement
- plan.markFailedMeasurementInsertion(i, mismatchException);
- }
- }
- } catch (MetadataException e) {
- if (config.isClusterMode()) {
- logger.debug(
- "meet error when check {}.{}, message: {}",
- devicePath,
- measurementList[i],
- e.getMessage());
- } else {
- logger.warn(
- "meet error when check {}.{}, message: {}",
- devicePath,
- measurementList[i],
- e.getMessage());
- }
- if (config.isEnablePartialInsert()) {
- // mark failed measurement
- plan.markFailedMeasurementInsertion(i, e);
- } else {
- throw e;
- }
- }
- }
-
- // set reusable device id
- plan.setDeviceID(deviceEntry.getDeviceID());
- // change device path to device id string for insertion
- plan.setDevicePath(new PartialPath(deviceEntry.getDeviceID().toStringID(), false));
-
- return deviceEntry.getDeviceID();
- }
-
/**
* register trigger to the timeseries
*
@@ -389,68 +317,6 @@ public class IDTableHashmapImpl implements IDTable {
return getIDiskSchemaManager().getDiskSchemaEntriesByOffset(offsets);
}
- /**
- * check whether a time series is exist if exist, check the type consistency if not exist, call
- * SchemaProcessor to create it
- *
- * @return measurement MNode of the time series or null if type is not match
- */
- private IMeasurementMNode getOrCreateMeasurementIfNotExist(
- DeviceEntry deviceEntry, InsertPlan plan, int loc) throws MetadataException {
- String measurementName = plan.getMeasurements()[loc];
- PartialPath seriesKey = new PartialPath(plan.getDevicePath().toString(), measurementName);
-
- SchemaEntry schemaEntry = deviceEntry.getSchemaEntry(measurementName);
-
- // if not exist, we create it
- if (schemaEntry == null) {
- // we have to copy plan's mnode for using id table's last cache
- IMeasurementMNode[] insertPlanMNodeBackup =
- new IMeasurementMNode[plan.getMeasurementMNodes().length];
- System.arraycopy(
- plan.getMeasurementMNodes(), 0, insertPlanMNodeBackup, 0, insertPlanMNodeBackup.length);
- try {
- IoTDB.schemaProcessor.getSeriesSchemasAndReadLockDevice(plan);
- } catch (IOException e) {
- throw new MetadataException(e);
- }
-
- // if the timeseries is in template, SchemaProcessor will not create timeseries. so we have to
- // put it
- // in id table here
- for (IMeasurementMNode measurementMNode : plan.getMeasurementMNodes()) {
- if (measurementMNode != null && !deviceEntry.contains(measurementMNode.getName())) {
- IMeasurementSchema schema = measurementMNode.getSchema();
- SchemaEntry curEntry =
- new SchemaEntry(
- schema.getType(),
- schema.getEncodingType(),
- schema.getCompressor(),
- deviceEntry.getDeviceID(),
- seriesKey,
- deviceEntry.isAligned(),
- IDiskSchemaManager);
- deviceEntry.putSchemaEntry(measurementMNode.getName(), curEntry);
- }
- }
-
- // copy back measurement mnode list
- System.arraycopy(
- insertPlanMNodeBackup, 0, plan.getMeasurementMNodes(), 0, insertPlanMNodeBackup.length);
-
- schemaEntry = deviceEntry.getSchemaEntry(measurementName);
- }
-
- // timeseries is using trigger, we should get trigger from SchemaProcessor
- if (schemaEntry.isUsingTrigger()) {
- IMeasurementMNode measurementMNode = IoTDB.schemaProcessor.getMeasurementMNode(seriesKey);
- return new InsertMeasurementMNode(
- measurementName, schemaEntry, measurementMNode.getTriggerExecutor());
- }
-
- return new InsertMeasurementMNode(measurementName, schemaEntry);
- }
-
/**
* get device id from device path and check is aligned,
*
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/ISchemaRegion.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/ISchemaRegion.java
index e0319ee600..6979bbffa8 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/ISchemaRegion.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/ISchemaRegion.java
@@ -41,7 +41,6 @@ import org.apache.iotdb.db.metadata.plan.schemaregion.write.ISetTemplatePlan;
import org.apache.iotdb.db.metadata.plan.schemaregion.write.IUnsetTemplatePlan;
import org.apache.iotdb.db.metadata.template.Template;
import org.apache.iotdb.db.mpp.common.schematree.DeviceSchemaInfo;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
import org.apache.iotdb.db.query.context.QueryContext;
@@ -396,8 +395,6 @@ public interface ISchemaRegion {
// endregion
// region Interfaces for InsertPlan process
- /** get schema for device. Attention!!! Only support insertPlan */
- IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan) throws MetadataException, IOException;
DeviceSchemaInfo getDeviceSchemaInfoWithAutoCreate(
PartialPath devicePath,
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java
index 040250dc4c..5b3458f1e0 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java
@@ -30,8 +30,6 @@ import org.apache.iotdb.consensus.ConsensusFactory;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.trigger.executor.TriggerEngine;
-import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
-import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
import org.apache.iotdb.db.exception.metadata.MeasurementAlreadyExistException;
import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
@@ -54,7 +52,6 @@ import org.apache.iotdb.db.metadata.mnode.IEntityMNode;
import org.apache.iotdb.db.metadata.mnode.IMNode;
import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
import org.apache.iotdb.db.metadata.mtree.MTreeBelowSGMemoryImpl;
import org.apache.iotdb.db.metadata.plan.schemaregion.ISchemaRegionPlan;
import org.apache.iotdb.db.metadata.plan.schemaregion.SchemaRegionPlanVisitor;
@@ -83,9 +80,6 @@ import org.apache.iotdb.db.metadata.template.Template;
import org.apache.iotdb.db.metadata.template.TemplateManager;
import org.apache.iotdb.db.mpp.common.schematree.DeviceSchemaInfo;
import org.apache.iotdb.db.mpp.common.schematree.MeasurementSchemaInfo;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
import org.apache.iotdb.db.query.context.QueryContext;
@@ -111,7 +105,6 @@ import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
@@ -1622,199 +1615,6 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
// endregion
// region Interfaces and Implementation for InsertPlan process
- /** get schema for device. Attention!!! Only support insertPlan */
- @Override
- @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
- public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
- throws MetadataException, IOException {
- // devicePath is a logical path which is parent of measurement, whether in template or not
- PartialPath devicePath = plan.getDevicePath();
- String[] measurementList = plan.getMeasurements();
- IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
- IMNode deviceMNode;
-
- // 1. get device node, set using template if accessed.
- deviceMNode = getDeviceInTemplateIfUsingTemplate(devicePath, measurementList);
- boolean isDeviceInTemplate = deviceMNode != null;
-
- // get logical device node, may be in template. will be multiple if overlap is allowed.
- if (!isDeviceInTemplate) {
- deviceMNode = getDeviceNodeWithAutoCreate(devicePath);
- }
-
- // check insert non-aligned InsertPlan for aligned timeseries
- if (deviceMNode.isEntity()) {
- if (plan.isAligned()) {
- if (!deviceMNode.getAsEntityMNode().isAligned()) {
- throw new AlignedTimeseriesException(
- "timeseries under this device are not aligned, " + "please use non-aligned interface",
- devicePath.getFullPath());
- }
- } else {
- if (deviceMNode.getAsEntityMNode().isAligned()) {
- throw new AlignedTimeseriesException(
- "timeseries under this device are aligned, " + "please use aligned interface",
- devicePath.getFullPath());
- }
- }
- }
-
- // 2. get schema of each measurement
- IMeasurementMNode measurementMNode;
- for (int i = 0; i < measurementList.length; i++) {
- try {
- // get MeasurementMNode, auto create if absent
- Pair<IMNode, IMeasurementMNode> pair =
- getMeasurementMNodeForInsertPlan(plan, i, deviceMNode, isDeviceInTemplate);
- deviceMNode = pair.left;
- measurementMNode = pair.right;
-
- // check type is match
- if (plan instanceof InsertRowPlan || plan instanceof InsertTabletPlan) {
- try {
- SchemaRegionUtils.checkDataTypeMatch(plan, i, measurementMNode.getSchema().getType());
- } catch (DataTypeMismatchException mismatchException) {
- logger.warn(mismatchException.getMessage());
- if (!config.isEnablePartialInsert()) {
- throw mismatchException;
- } else {
- // mark failed measurement
- plan.markFailedMeasurementInsertion(i, mismatchException);
- continue;
- }
- }
- measurementMNodes[i] = measurementMNode;
- // set measurementName instead of alias
- measurementList[i] = measurementMNode.getName();
- }
- } catch (MetadataException e) {
- if (config.isClusterMode()) {
- logger.debug(
- "meet error when check {}.{}, message: {}",
- devicePath,
- measurementList[i],
- e.getMessage());
- } else {
- logger.warn(
- "meet error when check {}.{}, message: {}",
- devicePath,
- measurementList[i],
- e.getMessage());
- }
- if (config.isEnablePartialInsert()) {
- // mark failed measurement
- plan.markFailedMeasurementInsertion(i, e);
- } else {
- throw e;
- }
- }
- }
-
- return deviceMNode;
- }
-
- private IMNode getDeviceInTemplateIfUsingTemplate(
- PartialPath devicePath, String[] measurementList) throws MetadataException, IOException {
- // 1. get device node, set using template if accessed.
- IMNode deviceMNode = null;
-
- // check every measurement path
- int index = mtree.getMountedNodeIndexOnMeasurementPath(devicePath, measurementList);
- if (index == devicePath.getNodeLength()) {
- return null;
- }
-
- // this measurement is in template, need to assure mounted node exists and set using
- // template.
- // Without allowing overlap of template and MTree, this block run only once
- String[] mountedPathNodes = Arrays.copyOfRange(devicePath.getNodes(), 0, index + 1);
- IMNode mountedNode = getDeviceNodeWithAutoCreate(new PartialPath(mountedPathNodes));
- if (!mountedNode.isUseTemplate()) {
- mountedNode = setUsingSchemaTemplate(mountedNode);
- }
- if (index < devicePath.getNodeLength() - 1) {
- deviceMNode =
- mountedNode
- .getUpperTemplate()
- .getPathNodeInTemplate(
- new PartialPath(
- Arrays.copyOfRange(
- devicePath.getNodes(), index + 1, devicePath.getNodeLength())));
- }
-
- return deviceMNode;
- }
-
- private Pair<IMNode, IMeasurementMNode> getMeasurementMNodeForInsertPlan(
- InsertPlan plan, int loc, IMNode deviceMNode, boolean isDeviceInTemplate)
- throws MetadataException {
- PartialPath devicePath = plan.getDevicePath();
- String[] measurementList = plan.getMeasurements();
- String measurement = measurementList[loc];
- IMeasurementMNode measurementMNode = null;
- if (isDeviceInTemplate) {
- measurementMNode = deviceMNode.getChild(measurement).getAsMeasurementMNode();
- } else {
- measurementMNode = getMeasurementMNode(deviceMNode, measurement);
- if (measurementMNode == null) {
- measurementMNode = findMeasurementInTemplate(deviceMNode, measurement);
- }
- }
- if (measurementMNode == null) {
- if (!config.isAutoCreateSchemaEnabled() || isDeviceInTemplate) {
- throw new PathNotExistException(devicePath + PATH_SEPARATOR + measurement);
- } else {
- if (plan instanceof InsertRowPlan || plan instanceof InsertTabletPlan) {
- if (!plan.isAligned()) {
- internalCreateTimeseries(devicePath.concatNode(measurement), plan.getDataTypes()[loc]);
- } else {
- internalAlignedCreateTimeseries(devicePath, measurement, plan.getDataTypes()[loc]);
- }
- // after creating timeseries, the deviceMNode has been replaced by a new entityMNode
- deviceMNode = mtree.getNodeByPath(devicePath);
- measurementMNode = getMeasurementMNode(deviceMNode, measurement);
- } else {
- throw new MetadataException(
- String.format(
- "Only support insertRow and insertTablet, plan is [%s]", plan.getOperatorType()));
- }
- }
- }
- return new Pair<>(deviceMNode, measurementMNode);
- }
-
- /** get dataType of plan, in loc measurements only support InsertRowPlan and InsertTabletPlan */
- private IMeasurementMNode findMeasurementInTemplate(IMNode deviceMNode, String measurement)
- throws MetadataException {
- Template curTemplate = deviceMNode.getUpperTemplate();
-
- if (curTemplate == null) {
- return null;
- }
-
- IMeasurementSchema schema = curTemplate.getSchema(measurement);
-
- if (schema == null) {
- return null;
- }
-
- if (!deviceMNode.isUseTemplate()) {
- deviceMNode = setUsingSchemaTemplate(deviceMNode);
- }
-
- return MeasurementMNode.getMeasurementMNode(
- deviceMNode.getAsEntityMNode(), measurement, schema, null);
- }
-
- /** create timeseries ignoring PathAlreadyExistException */
- private void internalCreateTimeseries(PartialPath path, TSDataType dataType)
- throws MetadataException {
- internalCreateTimeseries(
- path,
- dataType,
- getDefaultEncoding(dataType),
- TSFileDescriptor.getInstance().getConfig().getCompressor());
- }
/** create timeseries ignoring MeasurementAlreadyExistException */
private void internalCreateTimeseries(
@@ -1863,17 +1663,6 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
}
}
- /** create aligned timeseries ignoring MeasurementAlreadyExistException */
- private void internalAlignedCreateTimeseries(
- PartialPath devicePath, String measurement, TSDataType dataType) throws MetadataException {
- internalAlignedCreateTimeseries(
- devicePath,
- measurement,
- dataType,
- getDefaultEncoding(dataType),
- TSFileDescriptor.getInstance().getConfig().getCompressor());
- }
-
@Override
public DeviceSchemaInfo getDeviceSchemaInfoWithAutoCreate(
PartialPath devicePath,
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java
index d962671250..ae3ed43e02 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java
@@ -30,8 +30,6 @@ import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.trigger.executor.TriggerEngine;
import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
-import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
-import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
import org.apache.iotdb.db.exception.metadata.PathNotExistException;
@@ -52,7 +50,6 @@ import org.apache.iotdb.db.metadata.logfile.SchemaLogWriter;
import org.apache.iotdb.db.metadata.mnode.IMNode;
import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
import org.apache.iotdb.db.metadata.mtree.MTreeBelowSGCachedImpl;
import org.apache.iotdb.db.metadata.plan.schemaregion.ISchemaRegionPlan;
import org.apache.iotdb.db.metadata.plan.schemaregion.SchemaRegionPlanVisitor;
@@ -78,9 +75,6 @@ import org.apache.iotdb.db.metadata.tag.TagManager;
import org.apache.iotdb.db.metadata.template.Template;
import org.apache.iotdb.db.metadata.template.TemplateManager;
import org.apache.iotdb.db.mpp.common.schematree.DeviceSchemaInfo;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
import org.apache.iotdb.db.query.context.QueryContext;
@@ -88,7 +82,6 @@ import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
import org.apache.iotdb.db.utils.SchemaUtils;
import org.apache.iotdb.external.api.ISeriesNumerMonitor;
-import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
@@ -106,7 +99,6 @@ import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
@@ -118,7 +110,6 @@ import java.util.function.Function;
import java.util.stream.Stream;
import static java.util.stream.Collectors.toList;
-import static org.apache.iotdb.db.utils.EncodingInferenceUtils.getDefaultEncoding;
import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
/**
@@ -1749,101 +1740,6 @@ public class SchemaRegionSchemaFileImpl implements ISchemaRegion {
// endregion
// region Interfaces and Implementation for InsertPlan process
- /** get schema for device. Attention!!! Only support insertPlan */
- @Override
- @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
- public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
- throws MetadataException, IOException {
- // devicePath is a logical path which is parent of measurement, whether in template or not
- PartialPath devicePath = plan.getDevicePath();
- String[] measurementList = plan.getMeasurements();
- IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
- IMNode deviceMNode;
-
- // 1. get device node, set using template if accessed.
- deviceMNode = getDeviceInTemplateIfUsingTemplate(devicePath, measurementList);
- boolean isDeviceInTemplate = deviceMNode != null;
- // get logical device node, may be in template. will be multiple if overlap is allowed.
- if (!isDeviceInTemplate) {
- deviceMNode = getDeviceNodeWithAutoCreate(devicePath);
- }
- try {
- // check insert non-aligned InsertPlan for aligned timeseries
- if (deviceMNode.isEntity()) {
- if (plan.isAligned()) {
- if (!deviceMNode.getAsEntityMNode().isAligned()) {
- throw new AlignedTimeseriesException(
- "timeseries under this device are not aligned, "
- + "please use non-aligned interface",
- devicePath.getFullPath());
- }
- } else {
- if (deviceMNode.getAsEntityMNode().isAligned()) {
- throw new AlignedTimeseriesException(
- "timeseries under this device are aligned, " + "please use aligned interface",
- devicePath.getFullPath());
- }
- }
- }
-
- // 2. get schema of each measurement
- IMeasurementMNode measurementMNode;
- for (int i = 0; i < measurementList.length; i++) {
- try {
- // get MeasurementMNode, auto create if absent
- Pair<IMNode, IMeasurementMNode> pair =
- getMeasurementMNodeForInsertPlan(plan, i, deviceMNode, isDeviceInTemplate);
- deviceMNode = pair.left;
- measurementMNode = pair.right;
-
- // check type is match
- if (plan instanceof InsertRowPlan || plan instanceof InsertTabletPlan) {
- try {
- SchemaRegionUtils.checkDataTypeMatch(plan, i, measurementMNode.getSchema().getType());
- } catch (DataTypeMismatchException mismatchException) {
- logger.warn(mismatchException.getMessage());
- if (!config.isEnablePartialInsert()) {
- throw mismatchException;
- } else {
- // mark failed measurement
- plan.markFailedMeasurementInsertion(i, mismatchException);
- continue;
- }
- }
- measurementMNodes[i] = measurementMNode;
- // set measurementName instead of alias
- measurementList[i] = measurementMNode.getName();
- }
- } catch (MetadataException e) {
- if (config.isClusterMode()) {
- logger.debug(
- "meet error when check {}.{}, message: {}",
- devicePath,
- measurementList[i],
- e.getMessage());
- } else {
- logger.warn(
- "meet error when check {}.{}, message: {}",
- devicePath,
- measurementList[i],
- e.getMessage());
- }
- if (config.isEnablePartialInsert()) {
- // mark failed measurement
- plan.markFailedMeasurementInsertion(i, e);
- } else {
- throw e;
- }
- }
- }
- } finally {
- if (!isDeviceInTemplate) {
- mtree.unPinMNode(deviceMNode);
- }
- }
-
- return deviceMNode;
- }
@Override
public DeviceSchemaInfo getDeviceSchemaInfoWithAutoCreate(
@@ -1857,132 +1753,6 @@ public class SchemaRegionSchemaFileImpl implements ISchemaRegion {
throw new UnsupportedOperationException();
}
- private IMNode getDeviceInTemplateIfUsingTemplate(
- PartialPath devicePath, String[] measurementList) throws MetadataException, IOException {
- // 1. get device node, set using template if accessed.
- IMNode deviceMNode = null;
-
- // check every measurement path
- int index = mtree.getMountedNodeIndexOnMeasurementPath(devicePath, measurementList);
- if (index == devicePath.getNodeLength()) {
- return null;
- }
-
- // this measurement is in template, need to assure mounted node exists and set using
- // template.
- // Without allowing overlap of template and MTree, this block run only once
- String[] mountedPathNodes = Arrays.copyOfRange(devicePath.getNodes(), 0, index + 1);
- IMNode mountedNode = getDeviceNodeWithAutoCreate(new PartialPath(mountedPathNodes));
- try {
- if (!mountedNode.isUseTemplate()) {
- mountedNode = setUsingSchemaTemplate(mountedNode);
- }
- } finally {
- mtree.unPinMNode(mountedNode);
- }
-
- if (index < devicePath.getNodeLength() - 1) {
- deviceMNode =
- mountedNode
- .getUpperTemplate()
- .getPathNodeInTemplate(
- new PartialPath(
- Arrays.copyOfRange(
- devicePath.getNodes(), index + 1, devicePath.getNodeLength())));
- }
-
- return deviceMNode;
- }
-
- private Pair<IMNode, IMeasurementMNode> getMeasurementMNodeForInsertPlan(
- InsertPlan plan, int loc, IMNode deviceMNode, boolean isDeviceInTemplate)
- throws MetadataException {
- PartialPath devicePath = plan.getDevicePath();
- String[] measurementList = plan.getMeasurements();
- String measurement = measurementList[loc];
- IMeasurementMNode measurementMNode = null;
- if (isDeviceInTemplate) {
- measurementMNode = deviceMNode.getChild(measurement).getAsMeasurementMNode();
- } else {
- measurementMNode = getMeasurementMNode(deviceMNode, measurement);
- if (measurementMNode == null) {
- measurementMNode = findMeasurementInTemplate(deviceMNode, measurement);
- }
- }
- if (measurementMNode == null) {
- if (!config.isAutoCreateSchemaEnabled() || isDeviceInTemplate) {
- throw new PathNotExistException(devicePath + PATH_SEPARATOR + measurement);
- } else {
- if (plan instanceof InsertRowPlan || plan instanceof InsertTabletPlan) {
- if (!plan.isAligned()) {
- internalCreateTimeseries(devicePath.concatNode(measurement), plan.getDataTypes()[loc]);
- } else {
- internalAlignedCreateTimeseries(
- devicePath,
- Collections.singletonList(measurement),
- Collections.singletonList(plan.getDataTypes()[loc]));
- }
- // after creating timeseries, the deviceMNode has been replaced by a new entityMNode
- deviceMNode = mtree.getNodeByPath(devicePath);
- mtree.unPinMNode(deviceMNode);
- measurementMNode = getMeasurementMNode(deviceMNode, measurement);
- } else {
- throw new MetadataException(
- String.format(
- "Only support insertRow and insertTablet, plan is [%s]", plan.getOperatorType()));
- }
- }
- }
- return new Pair<>(deviceMNode, measurementMNode);
- }
-
- /** get dataType of plan, in loc measurements only support InsertRowPlan and InsertTabletPlan */
- private IMeasurementMNode findMeasurementInTemplate(IMNode deviceMNode, String measurement)
- throws MetadataException {
- Template curTemplate = deviceMNode.getUpperTemplate();
-
- if (curTemplate == null) {
- return null;
- }
-
- IMeasurementSchema schema = curTemplate.getSchema(measurement);
-
- if (schema == null) {
- return null;
- }
-
- if (!deviceMNode.isUseTemplate()) {
- deviceMNode = setUsingSchemaTemplate(deviceMNode);
- }
-
- return MeasurementMNode.getMeasurementMNode(
- deviceMNode.getAsEntityMNode(), measurement, schema, null);
- }
-
- /** create timeseries ignoring PathAlreadyExistException */
- private void internalCreateTimeseries(PartialPath path, TSDataType dataType)
- throws MetadataException {
- createTimeseries(
- path,
- dataType,
- getDefaultEncoding(dataType),
- TSFileDescriptor.getInstance().getConfig().getCompressor(),
- Collections.emptyMap());
- }
-
- /** create aligned timeseries ignoring PathAlreadyExistException */
- private void internalAlignedCreateTimeseries(
- PartialPath prefixPath, List<String> measurements, List<TSDataType> dataTypes)
- throws MetadataException {
- List<TSEncoding> encodings = new ArrayList<>();
- List<CompressionType> compressors = new ArrayList<>();
- for (TSDataType dataType : dataTypes) {
- encodings.add(getDefaultEncoding(dataType));
- compressors.add(TSFileDescriptor.getInstance().getConfig().getCompressor());
- }
- createAlignedTimeSeries(prefixPath, measurements, dataTypes, encodings, compressors);
- }
-
// endregion
// region Interfaces and Implementation for Template operations
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionUtils.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionUtils.java
index c95ea5b7f8..2a8d972b5f 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionUtils.java
@@ -21,12 +21,6 @@ package org.apache.iotdb.db.metadata.schemaregion;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.file.SystemFileFactory;
-import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
-import org.apache.iotdb.db.utils.TypeInferenceUtils;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.slf4j.Logger;
@@ -66,46 +60,4 @@ public class SchemaRegionUtils {
storageGroupDir.delete();
}
}
-
- public static void checkDataTypeMatch(InsertPlan plan, int loc, TSDataType dataTypeInSchema)
- throws MetadataException {
- TSDataType insertDataType;
- if (plan instanceof InsertRowPlan) {
- if (!((InsertRowPlan) plan).isNeedInferType()) {
- // only when InsertRowPlan's values is object[], we should check type
- insertDataType = getTypeInLoc(plan, loc);
- } else {
- insertDataType = dataTypeInSchema;
- }
- } else {
- insertDataType = getTypeInLoc(plan, loc);
- }
- if (dataTypeInSchema != insertDataType) {
- String measurement = plan.getMeasurements()[loc];
- String device = plan.getDevicePath().getFullPath();
- throw new DataTypeMismatchException(
- device,
- measurement,
- insertDataType,
- dataTypeInSchema,
- plan.getMinTime(),
- plan.getFirstValueOfIndex(loc));
- }
- }
-
- private static TSDataType getTypeInLoc(InsertPlan plan, int loc) throws MetadataException {
- TSDataType dataType;
- if (plan instanceof InsertRowPlan) {
- InsertRowPlan tPlan = (InsertRowPlan) plan;
- dataType =
- TypeInferenceUtils.getPredictedDataType(tPlan.getValues()[loc], tPlan.isNeedInferType());
- } else if (plan instanceof InsertTabletPlan) {
- dataType = (plan).getDataTypes()[loc];
- } else {
- throw new MetadataException(
- String.format(
- "Only support insert and insertTablet, plan is [%s]", plan.getOperatorType()));
- }
- return dataType;
- }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/write/InsertTabletNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/write/InsertTabletNode.java
index 606ee5dd93..c0fec29dfd 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/write/InsertTabletNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/write/InsertTabletNode.java
@@ -81,7 +81,7 @@ public class InsertTabletNode extends InsertNode implements WALEntryValue {
private int rowCount = 0;
- // when this plan is sub-plan split from another InsertTabletPlan, this indicates the original
+ // when this plan is sub-plan split from another InsertTabletNode, this indicates the original
// positions of values in
// this plan. For example, if the plan contains 5 timestamps, and range = [1,4,10,12], then it
// means that the first 3
diff --git a/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/dto/IoTDBPoint.java b/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/dto/IoTDBPoint.java
index a19f3682fa..57f27cb2f4 100644
--- a/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/dto/IoTDBPoint.java
+++ b/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/dto/IoTDBPoint.java
@@ -18,11 +18,7 @@
*/
package org.apache.iotdb.db.protocol.influxdb.dto;
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.protocol.influxdb.meta.IInfluxDBMetaManager;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
import org.apache.iotdb.db.utils.DataTypeUtils;
import org.apache.iotdb.db.utils.ParameterUtils;
import org.apache.iotdb.rpc.IoTDBConnectionException;
@@ -147,16 +143,6 @@ public class IoTDBPoint {
return values;
}
- public InsertRowPlan convertToInsertRowPlan()
- throws IllegalPathException, IoTDBConnectionException, QueryProcessException {
- return new InsertRowPlan(
- new PartialPath(getDeviceId()),
- getTime(),
- getMeasurements().toArray(new String[0]),
- DataTypeUtils.getValueBuffer(getTypes(), getValues()),
- false);
- }
-
/**
* Convert IoTDB point to InsertRecordReq
*
diff --git a/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/meta/TagInfoRecords.java b/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/meta/TagInfoRecords.java
index c38df89d0f..d604956429 100644
--- a/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/meta/TagInfoRecords.java
+++ b/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/meta/TagInfoRecords.java
@@ -18,17 +18,11 @@
*/
package org.apache.iotdb.db.protocol.influxdb.meta;
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
import org.apache.iotdb.db.utils.DataTypeUtils;
import org.apache.iotdb.rpc.IoTDBConnectionException;
import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.influxdb.InfluxDBException;
-
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicLong;
@@ -81,24 +75,6 @@ public class TagInfoRecords {
valuesList.add(values);
}
- public List<InsertRowPlan> convertToInsertRowPlans() {
- ArrayList<InsertRowPlan> insertRowPlans = new ArrayList<>();
- for (int i = 0; i < deviceIds.size(); i++) {
- try {
- insertRowPlans.add(
- new InsertRowPlan(
- new PartialPath(deviceIds.get(i)),
- times.get(i),
- measurementsList.get(i).toArray(new String[0]),
- DataTypeUtils.getValueBuffer(typesList.get(i), valuesList.get(i)),
- false));
- } catch (QueryProcessException | IllegalPathException | IoTDBConnectionException e) {
- throw new InfluxDBException(e.getMessage());
- }
- }
- return insertRowPlans;
- }
-
public List<TSInsertRecordReq> convertToInsertRecordsReq(long sessionID)
throws IoTDBConnectionException {
ArrayList<TSInsertRecordReq> reqs = new ArrayList<>();
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/DeleteDataOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/DeleteDataOperator.java
index d424985b9b..a0898ed1e9 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/DeleteDataOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/DeleteDataOperator.java
@@ -22,15 +22,11 @@ import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.qp.logical.Operator;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
import java.util.ArrayList;
-import java.util.LinkedList;
import java.util.List;
-import static org.apache.iotdb.commons.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
-
/** this class extends {@code RootOperator} and process delete statement. */
public class DeleteDataOperator extends Operator {
@@ -72,15 +68,6 @@ public class DeleteDataOperator extends Operator {
@Override
public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
throws QueryProcessException {
- List<PartialPath> originPath = getPaths();
- if (isPrefixMatchPath()) {
- // adapt to prefix match of 0.12
- List<PartialPath> addedPath = new LinkedList<>();
- for (PartialPath path : originPath) {
- addedPath.add(path.concatNode(MULTI_LEVEL_PATH_WILDCARD));
- }
- originPath.addAll(addedPath);
- }
- return new DeletePlan(getStartTime(), getEndTime(), originPath);
+ return null;
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InsertOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InsertOperator.java
index adc3988088..12f43e9110 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InsertOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InsertOperator.java
@@ -20,11 +20,8 @@ package org.apache.iotdb.db.qp.logical.crud;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.exception.sql.SQLParserException;
import org.apache.iotdb.db.qp.logical.Operator;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowsPlan;
import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
import java.util.List;
@@ -88,32 +85,6 @@ public class InsertOperator extends Operator {
@Override
public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
throws QueryProcessException {
- int measurementsNum = measurementList.length;
- if (times.length == 1) {
- if (measurementsNum != valueLists.get(0).length) {
- throw new SQLParserException(
- String.format(
- "the measurementList's size %d is not consistent with the valueList's size %d",
- measurementsNum, valueLists.get(0).length));
- }
- InsertRowPlan insertRowPlan =
- new InsertRowPlan(device, times[0], measurementList, valueLists.get(0));
- insertRowPlan.setAligned(isAligned);
- return insertRowPlan;
- }
- InsertRowsPlan insertRowsPlan = new InsertRowsPlan();
- for (int i = 0; i < times.length; i++) {
- if (measurementsNum != valueLists.get(i).length) {
- throw new SQLParserException(
- String.format(
- "the measurementList's size %d is not consistent with the valueList's size %d",
- measurementsNum, valueLists.get(i).length));
- }
- InsertRowPlan insertRowPlan =
- new InsertRowPlan(device, times[i], measurementList.clone(), valueLists.get(i));
- insertRowPlan.setAligned(isAligned);
- insertRowsPlan.addOneInsertRowPlan(insertRowPlan, i);
- }
- return insertRowsPlan;
+ return null;
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DeletePartitionOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DeletePartitionOperator.java
index 8b0e673970..e026e62beb 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DeletePartitionOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DeletePartitionOperator.java
@@ -23,7 +23,6 @@ import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.qp.logical.Operator;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.DeletePartitionPlan;
import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
import java.util.Set;
@@ -57,6 +56,6 @@ public class DeletePartitionOperator extends Operator {
@Override
public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
throws QueryProcessException {
- return new DeletePartitionPlan(storageGroupName, partitionIds);
+ return null;
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
index 39ef94b632..52df1bef5f 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
@@ -25,12 +25,6 @@ import org.apache.iotdb.consensus.common.request.IConsensusRequest;
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.qp.logical.Operator;
import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertMultiTabletsPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowsOfOneDevicePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowsPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
import org.apache.iotdb.db.qp.physical.crud.SelectIntoPlan;
import org.apache.iotdb.db.qp.physical.sys.ActivateTemplateInClusterPlan;
import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
@@ -313,18 +307,6 @@ public abstract class PhysicalPlan implements IConsensusRequest {
PhysicalPlan plan;
// TODO-Cluster: support more plans
switch (type) {
- case INSERT:
- plan = new InsertRowPlan();
- break;
- case BATCHINSERT:
- plan = new InsertTabletPlan();
- break;
- case MULTI_BATCH_INSERT:
- plan = new InsertMultiTabletsPlan();
- break;
- case DELETE:
- plan = new DeletePlan();
- break;
case SET_STORAGE_GROUP:
plan = new SetStorageGroupPlan();
break;
@@ -421,12 +403,6 @@ public abstract class PhysicalPlan implements IConsensusRequest {
case STORAGE_GROUP_MNODE:
plan = new StorageGroupMNodePlan();
break;
- case BATCH_INSERT_ROWS:
- plan = new InsertRowsPlan();
- break;
- case BATCH_INSERT_ONE_DEVICE:
- plan = new InsertRowsOfOneDevicePlan();
- break;
case CREATE_TRIGGER:
plan = new CreateTriggerPlan();
break;
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePartitionPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePartitionPlan.java
deleted file mode 100644
index 285d6d396a..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePartitionPlan.java
+++ /dev/null
@@ -1,53 +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.qp.physical.crud;
-
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-
-public class DeletePartitionPlan extends PhysicalPlan {
-
- private PartialPath storageGroupName;
- private Set<Long> partitionId;
-
- public DeletePartitionPlan(PartialPath storageGroupName, Set<Long> partitionId) {
- super(OperatorType.DELETE_PARTITION);
- this.storageGroupName = storageGroupName;
- this.partitionId = partitionId;
- }
-
- @Override
- public List<PartialPath> getPaths() {
- return Collections.emptyList();
- }
-
- public PartialPath getStorageGroupName() {
- return storageGroupName;
- }
-
- public Set<Long> getPartitionId() {
- return partitionId;
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java
deleted file mode 100644
index 74155eb30f..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java
+++ /dev/null
@@ -1,221 +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.qp.physical.crud;
-
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.engine.storagegroup.DataRegion.TimePartitionFilter;
-import org.apache.iotdb.db.qp.logical.Operator;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
-import org.apache.iotdb.db.wal.buffer.WALEntryValue;
-import org.apache.iotdb.db.wal.utils.WALWriteUtils;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-
-public class DeletePlan extends PhysicalPlan implements WALEntryValue {
- /** byte: type, integer: paths.size(), long: deleteStartTime, deleteEndTime, index */
- private static final int FIXED_SERIALIZED_SIZE = Byte.BYTES + Integer.BYTES + Long.BYTES * 3;
-
- private long deleteStartTime;
- private long deleteEndTime;
- private List<PartialPath> paths = new ArrayList<>();
- /**
- * This deletion only affects those time partitions that evaluate true by the filter. If the
- * filter is null, all partitions are processed. This is to avoid redundant data deletions when
- * one timeseries deletion is split and executed into different replication groups.
- */
- private TimePartitionFilter partitionFilter;
-
- public DeletePlan() {
- super(Operator.OperatorType.DELETE);
- }
-
- /**
- * constructor of DeletePlan with single path.
- *
- * @param startTime delete time range start
- * @param endTime delete time range end
- * @param path time series path
- */
- public DeletePlan(long startTime, long endTime, PartialPath path) {
- super(Operator.OperatorType.DELETE);
- this.deleteStartTime = startTime;
- this.deleteEndTime = endTime;
- this.paths.add(path);
- }
-
- /**
- * constructor of DeletePlan with multiple paths.
- *
- * @param startTime delete time range start
- * @param endTime delete time range end
- * @param paths time series paths in List structure
- */
- public DeletePlan(long startTime, long endTime, List<PartialPath> paths) {
- super(Operator.OperatorType.DELETE);
- this.deleteStartTime = startTime;
- this.deleteEndTime = endTime;
- this.paths = paths;
- }
-
- public long getDeleteStartTime() {
- return deleteStartTime;
- }
-
- public void setDeleteStartTime(long delTime) {
- this.deleteStartTime = delTime;
- }
-
- public long getDeleteEndTime() {
- return deleteEndTime;
- }
-
- public void setDeleteEndTime(long delTime) {
- this.deleteEndTime = delTime;
- }
-
- public void addPath(PartialPath path) {
- this.paths.add(path);
- }
-
- public void addPaths(List<PartialPath> paths) {
- this.paths.addAll(paths);
- }
-
- @Override
- public List<PartialPath> getPaths() {
- return paths;
- }
-
- @Override
- public void setPaths(List<PartialPath> paths) {
- this.paths = paths;
- }
-
- public TimePartitionFilter getPartitionFilter() {
- return partitionFilter;
- }
-
- public void setPartitionFilter(TimePartitionFilter partitionFilter) {
- this.partitionFilter = partitionFilter;
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(deleteStartTime, deleteEndTime, paths);
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- DeletePlan that = (DeletePlan) o;
- return deleteStartTime == that.deleteStartTime
- && deleteEndTime == that.deleteEndTime
- && Objects.equals(paths, that.paths);
- }
-
- @Override
- public int serializedSize() {
- int size = FIXED_SERIALIZED_SIZE;
- for (PartialPath path : paths) {
- size += ReadWriteIOUtils.sizeToWrite(path.getFullPath());
- }
- return size;
- }
-
- @Override
- public void serialize(DataOutputStream stream) throws IOException {
- int type = PhysicalPlanType.DELETE.ordinal();
- stream.writeByte((byte) type);
- stream.writeLong(deleteStartTime);
- stream.writeLong(deleteEndTime);
- stream.writeInt(paths.size());
- for (PartialPath path : paths) {
- putString(stream, path.getFullPath());
- }
-
- stream.writeLong(index);
- }
-
- @Override
- public void serializeImpl(ByteBuffer buffer) {
- int type = PhysicalPlanType.DELETE.ordinal();
- buffer.put((byte) type);
- buffer.putLong(deleteStartTime);
- buffer.putLong(deleteEndTime);
- buffer.putInt(paths.size());
- for (PartialPath path : paths) {
- putString(buffer, path.getFullPath());
- }
-
- buffer.putLong(index);
- }
-
- @Override
- public void serializeToWAL(IWALByteBufferView buffer) {
- int type = PhysicalPlanType.DELETE.ordinal();
- buffer.put((byte) type);
- buffer.putLong(deleteStartTime);
- buffer.putLong(deleteEndTime);
- buffer.putInt(paths.size());
- for (PartialPath path : paths) {
- WALWriteUtils.write(path.getFullPath(), buffer);
- }
- buffer.putLong(index);
- }
-
- @Override
- public void deserialize(DataInputStream stream) throws IOException, IllegalPathException {
- this.deleteStartTime = stream.readLong();
- this.deleteEndTime = stream.readLong();
- int pathSize = stream.readInt();
- this.paths = new ArrayList<>();
- for (int i = 0; i < pathSize; i++) {
- paths.add(new PartialPath(ReadWriteIOUtils.readString(stream)));
- }
-
- this.index = stream.readLong();
- }
-
- @Override
- public void deserialize(ByteBuffer buffer) throws IllegalPathException {
- this.deleteStartTime = buffer.getLong();
- this.deleteEndTime = buffer.getLong();
- int pathSize = buffer.getInt();
- this.paths = new ArrayList<>();
- for (int i = 0; i < pathSize; i++) {
- paths.add(new PartialPath(readString(buffer)));
- }
-
- this.index = buffer.getLong();
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertMultiTabletsPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertMultiTabletsPlan.java
deleted file mode 100644
index d653cfc83d..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertMultiTabletsPlan.java
+++ /dev/null
@@ -1,453 +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.qp.physical.crud;
-
-import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.commons.utils.StatusUtils;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
-import org.apache.iotdb.db.qp.physical.BatchPlan;
-import org.apache.iotdb.tsfile.exception.NotImplementedException;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.TreeMap;
-
-/**
- * Mainly used in the distributed version, when multiple InsertTabletPlans belong to a raft
- * replication group, we merge these InsertTabletPlans into one InsertMultiTabletsPlan, which can
- * reduce the number of raft logs. For details, please refer to
- * https://issues.apache.org/jira/browse/IOTDB-1099
- */
-public class InsertMultiTabletsPlan extends InsertPlan implements BatchPlan {
-
- /**
- * the value is used to indict the parent InsertTabletPlan's index when the parent
- * InsertTabletPlan is split to multi sub InsertTabletPlans. if the InsertTabletPlan have no
- * parent plan, the value is zero;
- *
- * <p>suppose we originally have three InsertTabletPlans in one InsertMultiTabletsPlan, then the
- * initial InsertMultiTabletsPlan would have the following two attributes:
- *
- * <p>insertTabletPlanList={InsertTabletPlan_1,InsertTabletPlan_2,InsertTabletPlan_3}
- *
- * <p>parentInsetTablePlanIndexList={0,0,0} both have three values.
- *
- * <p>if the InsertTabletPlan_1 is split into two sub InsertTabletPlans, InsertTabletPlan_2 is
- * split into three sub InsertTabletPlans, InsertTabletPlan_3 is split into four sub
- * InsertTabletPlans.
- *
- * <p>InsertTabletPlan_1={InsertTabletPlan_1_subPlan1, InsertTabletPlan_1_subPlan2}
- *
- * <p>InsertTabletPlan_2={InsertTabletPlan_2_subPlan1, InsertTabletPlan_2_subPlan2,
- * InsertTabletPlan_2_subPlan3}
- *
- * <p>InsertTabletPlan_3={InsertTabletPlan_3_subPlan1, InsertTabletPlan_3_subPlan2,
- * InsertTabletPlan_3_subPlan3, InsertTabletPlan_3_subPlan4}
- *
- * <p>those sub plans belong to two different raft data groups, so will generate two new
- * InsertMultiTabletPlans
- *
- * <p>InsertMultiTabletPlant1.insertTabletPlanList={InsertTabletPlan_1_subPlan1,
- * InsertTabletPlan_3_subPlan1, InsertTabletPlan_3_subPlan3, InsertTabletPlan_3_subPlan4}
- *
- * <p>InsertMultiTabletPlant1.parentInsetTablePlanIndexList={0,2,2,2}
- *
- * <p>InsertMultiTabletPlant2.insertTabletPlanList={InsertTabletPlan_1_subPlan2,
- * InsertTabletPlan_2_subPlan1, InsertTabletPlan_2_subPlan2, InsertTabletPlan_2_subPlan3,
- * InsertTabletPlan_3_subPlan2}
- *
- * <p>InsertMultiTabletPlant2.parentInsetTablePlanIndexList={0,1,1,1,2}
- *
- * <p>this is usually used to back-propagate exceptions to the parent plan without losing their
- * proper positions.
- */
- List<Integer> parentInsertTabletPlanIndexList;
-
- /** the InsertTabletPlan list */
- List<InsertTabletPlan> insertTabletPlanList;
-
- /** record the result of creation of time series */
- private Map<Integer, TSStatus> results = new TreeMap<>();
-
- private List<PartialPath> prefixPaths;
-
- boolean[] isExecuted;
-
- Boolean isEnableMultithreading;
-
- Integer differentStorageGroupsCount;
-
- public InsertMultiTabletsPlan() {
- super(OperatorType.MULTI_BATCH_INSERT);
- this.insertTabletPlanList = new ArrayList<>();
- this.parentInsertTabletPlanIndexList = new ArrayList<>();
- }
-
- public InsertMultiTabletsPlan(List<InsertTabletPlan> insertTabletPlanList) {
- super(OperatorType.MULTI_BATCH_INSERT);
- this.insertTabletPlanList = insertTabletPlanList;
- this.parentInsertTabletPlanIndexList = new ArrayList<>();
- }
-
- public InsertMultiTabletsPlan(
- List<InsertTabletPlan> insertTabletPlanList, List<Integer> parentInsertTabletPlanIndexList) {
- super(OperatorType.MULTI_BATCH_INSERT);
- this.insertTabletPlanList = insertTabletPlanList;
- this.parentInsertTabletPlanIndexList = parentInsertTabletPlanIndexList;
- }
-
- public void addInsertTabletPlan(InsertTabletPlan plan, Integer parentIndex) {
- insertTabletPlanList.add(plan);
- parentInsertTabletPlanIndexList.add(parentIndex);
- }
-
- @Override
- public List<PartialPath> getPaths() {
- List<PartialPath> result = new ArrayList<>();
- for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
- result.addAll(insertTabletPlan.getPaths());
- }
- return result;
- }
-
- @Override
- public List<PartialPath> getPrefixPaths() {
- if (prefixPaths != null) {
- return prefixPaths;
- }
- prefixPaths = new ArrayList<>(insertTabletPlanList.size());
- for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
- prefixPaths.add(insertTabletPlan.getDevicePath());
- }
- return prefixPaths;
- }
-
- @Override
- public long getMinTime() {
- long minTime = Long.MAX_VALUE;
- for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
- if (minTime > insertTabletPlan.getMinTime()) {
- minTime = insertTabletPlan.getMinTime();
- }
- }
- return minTime;
- }
-
- public long getMaxTime() {
- long maxTime = Long.MIN_VALUE;
- for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
- if (maxTime < insertTabletPlan.getMaxTime()) {
- maxTime = insertTabletPlan.getMaxTime();
- }
- }
- return maxTime;
- }
-
- @Override
- public Object getFirstValueOfIndex(int index) {
- throw new NotImplementedException();
- }
-
- public int getTabletsSize() {
- return insertTabletPlanList.size();
- }
-
- @Override
- public Map<Integer, TSStatus> getResults() {
- return results;
- }
-
- /** @return the total row of the whole InsertTabletPlan */
- public int getTotalRowCount() {
- int rowCount = 0;
- for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
- rowCount += insertTabletPlan.getRowCount();
- }
- return rowCount;
- }
-
- /**
- * Gets the number of rows in the InsertTabletPlan of the index, zero will be returned if the
- * index is out of bound.
- *
- * @param index the index of the insertTabletPlanList
- * @return the total row count of the insertTabletPlanList.get(i)
- */
- public int getRowCount(int index) {
- if (index >= insertTabletPlanList.size() || index < 0) {
- return 0;
- }
- return insertTabletPlanList.get(index).getRowCount();
- }
-
- public PartialPath getFirstDeviceId() {
- return insertTabletPlanList.get(0).getDevicePath();
- }
-
- public InsertTabletPlan getInsertTabletPlan(int index) {
- if (index >= insertTabletPlanList.size() || index < 0) {
- return null;
- }
- return insertTabletPlanList.get(index);
- }
-
- /**
- * @param index the index of the sub plan in this InsertMultiTabletsPlan
- * @return the parent's index in the parent InsertMultiTabletsPlan
- */
- public int getParentIndex(int index) {
- if (index >= parentInsertTabletPlanIndexList.size() || index < 0) {
- return -1;
- }
- return parentInsertTabletPlanIndexList.get(index);
- }
-
- @Override
- public void checkIntegrity() throws QueryProcessException {
- if (insertTabletPlanList.isEmpty()) {
- throw new QueryProcessException("sub tablet is empty.");
- }
- for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
- insertTabletPlan.checkIntegrity();
- }
- }
-
- public void setParentInsertTabletPlanIndexList(List<Integer> parentInsertTabletPlanIndexList) {
- this.parentInsertTabletPlanIndexList = parentInsertTabletPlanIndexList;
- }
-
- public List<Integer> getParentInsertTabletPlanIndexList() {
- return parentInsertTabletPlanIndexList;
- }
-
- public void setInsertTabletPlanList(List<InsertTabletPlan> insertTabletPlanList) {
- this.insertTabletPlanList = insertTabletPlanList;
- }
-
- public List<InsertTabletPlan> getInsertTabletPlanList() {
- return insertTabletPlanList;
- }
-
- public TSStatus[] getFailingStatus() {
- return StatusUtils.getFailingStatus(results, insertTabletPlanList.size());
- }
-
- public void setResults(Map<Integer, TSStatus> results) {
- this.results = results;
- }
-
- @Override
- public void recoverFromFailure() {
- for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
- insertTabletPlan.recoverFromFailure();
- }
- }
-
- @Override
- public void serializeImpl(ByteBuffer buffer) {
- int type = PhysicalPlanType.MULTI_BATCH_INSERT.ordinal();
- buffer.put((byte) type);
- buffer.putInt(insertTabletPlanList.size());
- for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
- insertTabletPlan.subSerialize(buffer, 0, insertTabletPlan.getRowCount());
- }
-
- buffer.putInt(parentInsertTabletPlanIndexList.size());
- for (Integer index : parentInsertTabletPlanIndexList) {
- buffer.putInt(index);
- }
- }
-
- @Override
- public void serialize(DataOutputStream stream) throws IOException {
- int type = PhysicalPlanType.MULTI_BATCH_INSERT.ordinal();
- stream.writeByte((byte) type);
- stream.writeInt(insertTabletPlanList.size());
- for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
- insertTabletPlan.subSerialize(stream, 0, insertTabletPlan.getRowCount());
- }
-
- stream.writeInt(parentInsertTabletPlanIndexList.size());
- for (Integer index : parentInsertTabletPlanIndexList) {
- stream.writeInt(index);
- }
- }
-
- @Override
- public void deserialize(ByteBuffer buffer) throws IllegalPathException {
- int tmpInsertTabletPlanListSize = buffer.getInt();
- this.insertTabletPlanList = new ArrayList<>(tmpInsertTabletPlanListSize);
- for (int i = 0; i < tmpInsertTabletPlanListSize; i++) {
- InsertTabletPlan tmpPlan = new InsertTabletPlan();
- tmpPlan.deserialize(buffer);
- this.insertTabletPlanList.add(tmpPlan);
- }
-
- int tmpParentInsetTablePlanIndexListSize = buffer.getInt();
- this.parentInsertTabletPlanIndexList = new ArrayList<>(tmpParentInsetTablePlanIndexListSize);
- for (int i = 0; i < tmpParentInsetTablePlanIndexListSize; i++) {
- this.parentInsertTabletPlanIndexList.add(buffer.getInt());
- }
- }
-
- @Override
- public void setIndex(long index) {
- super.setIndex(index);
- for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
- // use the InsertMultiTabletsPlan's index as the sub InsertTabletPlan's index
- insertTabletPlan.setIndex(index);
- }
- }
-
- @Override
- public String toString() {
- return "InsertMultiTabletsPlan{"
- + " insertTabletPlanList="
- + insertTabletPlanList
- + ", parentInsetTablePlanIndexList="
- + parentInsertTabletPlanIndexList
- + "}";
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- InsertMultiTabletsPlan that = (InsertMultiTabletsPlan) o;
-
- if (!Objects.equals(insertTabletPlanList, that.insertTabletPlanList)) {
- return false;
- }
- return Objects.equals(parentInsertTabletPlanIndexList, that.parentInsertTabletPlanIndexList);
- }
-
- @Override
- public int hashCode() {
- int result = insertTabletPlanList != null ? insertTabletPlanList.hashCode() : 0;
- result =
- 31 * result
- + (parentInsertTabletPlanIndexList != null
- ? parentInsertTabletPlanIndexList.hashCode()
- : 0);
- return result;
- }
-
- @Override
- public void setIsExecuted(int i) {
- if (isExecuted == null) {
- isExecuted = new boolean[getBatchSize()];
- }
- isExecuted[i] = true;
- }
-
- @Override
- public boolean isExecuted(int i) {
- if (isExecuted == null) {
- isExecuted = new boolean[getBatchSize()];
- }
- return isExecuted[i];
- }
-
- @Override
- public int getBatchSize() {
- return insertTabletPlanList.size();
- }
-
- @Override
- public void unsetIsExecuted(int i) {
- if (isExecuted == null) {
- isExecuted = new boolean[getBatchSize()];
- }
- isExecuted[i] = false;
- if (parentInsertTabletPlanIndexList != null && !parentInsertTabletPlanIndexList.isEmpty()) {
- results.remove(getParentIndex(i));
- } else {
- results.remove(i);
- }
- }
-
- public int getDifferentStorageGroupsCount() {
- if (differentStorageGroupsCount == null) {
- Set<String> insertPlanSGSet = new HashSet<>();
- int defaultStorageGroupLevel =
- IoTDBDescriptor.getInstance().getConfig().getDefaultStorageGroupLevel();
- for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
- String[] nodes = insertTabletPlan.getDevicePath().getNodes();
- StringBuilder stringBuilder = new StringBuilder();
- for (int i = 0; i <= defaultStorageGroupLevel && i < nodes.length; i++) {
- stringBuilder.append(nodes[i]).append(".");
- }
- insertPlanSGSet.add(stringBuilder.toString());
- }
- differentStorageGroupsCount = insertPlanSGSet.size();
- }
- return differentStorageGroupsCount;
- }
-
- public boolean isEnableMultiThreading() {
- // If we enable multithreading, we need to consider the loss of switching between threads,
- // so we need to judge the core threads of the thread pool and the size of the tablet.
-
- // Therefore, we set the number of core threads in the thread pool to
- // min(the number of different sg, availableProcessors()/2),
- // and need columns >= insertMultiTabletEnableMultithreadingColumnThreshold.
-
- // It should be noted that if the number of sg is large and exceeds twice of the recommended
- // number of CPU threads,
- // it may lead to failure to allocate out of heap memory and NPE.
- // Therefore, we will also turn off multithreading in this case.
- if (isEnableMultithreading == null) {
- int sgSize = getDifferentStorageGroupsCount();
- // SG should be >= 1 so that it will not be locked and degenerate into serial.
- // SG should be <= Runtime.getRuntime().availableProcessors()*2 so that to avoid failure to
- // allocate out of heap memory and NPE
- if (sgSize <= 1 || sgSize >= Runtime.getRuntime().availableProcessors() * 2) {
- isEnableMultithreading = false;
- } else {
- int count = 0;
- for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
- if (insertTabletPlan.getColumns().length
- >= IoTDBDescriptor.getInstance()
- .getConfig()
- .getInsertMultiTabletEnableMultithreadingColumnThreshold()) {
- count++;
- }
- }
- isEnableMultithreading = count * 2 >= insertTabletPlanList.size();
- }
- }
- return isEnableMultithreading;
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertPlan.java
deleted file mode 100644
index a5ef81ff12..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertPlan.java
+++ /dev/null
@@ -1,239 +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.qp.physical.crud;
-
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.qp.logical.Operator;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-public abstract class InsertPlan extends PhysicalPlan {
-
- /**
- * if use id table, this filed is id form of device path <br>
- * if not, this filed is device path<br>
- */
- protected PartialPath devicePath;
-
- protected boolean isAligned;
- protected String[] measurements;
- // get from client
- protected TSDataType[] dataTypes;
- // get from SchemaProcessor
- protected IMeasurementMNode[] measurementMNodes;
-
- /**
- * device id reference, for reuse device id in both id table and memtable <br>
- * used in memtable
- */
- protected IDeviceID deviceID;
-
- // record the failed measurements, their reasons, and positions in "measurements"
- List<String> failedMeasurements;
- private List<Exception> failedExceptions;
- List<Integer> failedIndices;
-
- public InsertPlan(Operator.OperatorType operatorType) {
- super(operatorType);
- super.canBeSplit = false;
- }
-
- /**
- * if use id table, this filed is id form of device path <br>
- * if not, this filed is device path<br>
- * used in flush time manager, last cache, tsfile processor
- */
- public PartialPath getDevicePath() {
- return devicePath;
- }
-
- public void setDevicePath(PartialPath devicePath) {
- this.devicePath = devicePath;
- }
-
- public String[] getMeasurements() {
- return this.measurements;
- }
-
- public void setMeasurements(String[] measurements) {
- this.measurements = measurements;
- }
-
- public boolean hasValidMeasurements() {
- for (Object o : measurements) {
- if (o != null) {
- return true;
- }
- }
- return false;
- }
-
- public TSDataType[] getDataTypes() {
- return dataTypes;
- }
-
- public void setDataTypes(TSDataType[] dataTypes) {
- this.dataTypes = dataTypes;
- }
-
- public IMeasurementMNode[] getMeasurementMNodes() {
- return measurementMNodes;
- }
-
- public void setMeasurementMNodes(IMeasurementMNode[] mNodes) {
- this.measurementMNodes = mNodes;
- }
-
- public List<String> getFailedMeasurements() {
- return failedMeasurements;
- }
-
- public List<Exception> getFailedExceptions() {
- return failedExceptions;
- }
-
- public int getFailedMeasurementNumber() {
- return failedMeasurements == null ? 0 : failedMeasurements.size();
- }
-
- public boolean isAligned() {
- return isAligned;
- }
-
- public void setAligned(boolean aligned) {
- isAligned = aligned;
- }
-
- public abstract long getMinTime();
-
- public abstract Object getFirstValueOfIndex(int index);
-
- /**
- * This method is overrided in InsertRowPlan and InsertTabletPlan. After marking failed
- * measurements, the failed values or columns would be null as well. We'd better use
- * "measurements[index] == null" to determine if the measurement failed.
- *
- * @param index failed measurement index
- */
- public void markFailedMeasurementInsertion(int index, Exception e) {
- if (measurements[index] == null) {
- return;
- }
- if (failedMeasurements == null) {
- failedMeasurements = new ArrayList<>();
- failedExceptions = new ArrayList<>();
- failedIndices = new ArrayList<>();
- }
- failedMeasurements.add(measurements[index]);
- failedExceptions.add(e);
- failedIndices.add(index);
- measurements[index] = null;
- }
-
- /**
- * Reconstruct this plan with the failed measurements.
- *
- * @return the plan itself, with measurements replaced with the previously failed ones.
- */
- public InsertPlan getPlanFromFailed() {
- if (failedMeasurements == null) {
- return null;
- }
- measurements = failedMeasurements.toArray(new String[0]);
- failedMeasurements = null;
- if (dataTypes != null) {
- TSDataType[] temp = dataTypes.clone();
- dataTypes = new TSDataType[failedIndices.size()];
- for (int i = 0; i < failedIndices.size(); i++) {
- dataTypes[i] = temp[failedIndices.get(i)];
- }
- }
- if (measurementMNodes != null) {
- IMeasurementMNode[] temp = measurementMNodes.clone();
- measurementMNodes = new IMeasurementMNode[failedIndices.size()];
- for (int i = 0; i < failedIndices.size(); i++) {
- measurementMNodes[i] = temp[failedIndices.get(i)];
- }
- }
-
- failedIndices = null;
- failedExceptions = null;
- return this;
- }
-
- /** Reset measurements from failed measurements (if any), as if no failure had ever happened. */
- public void recoverFromFailure() {
- if (failedMeasurements == null) {
- return;
- }
-
- for (int i = 0; i < failedMeasurements.size(); i++) {
- int index = failedIndices.get(i);
- measurements[index] = failedMeasurements.get(i);
- }
- failedIndices = null;
- failedExceptions = null;
- failedMeasurements = null;
- }
-
- @Override
- public void checkIntegrity() throws QueryProcessException {
- if (devicePath == null) {
- throw new QueryProcessException("DeviceId is null");
- }
- if (measurements == null) {
- throw new QueryProcessException("Measurements are null");
- }
- Set<String> deduplicatedMeasurements = new HashSet<>();
- for (String measurement : measurements) {
- if (measurement == null || measurement.isEmpty()) {
- throw new QueryProcessException(
- "Measurement contains null or empty string: " + Arrays.toString(measurements));
- }
- if (deduplicatedMeasurements.contains(measurement)) {
- throw new QueryProcessException(
- "Insertion contains duplicated measurement: " + measurement);
- } else {
- deduplicatedMeasurements.add(measurement);
- }
- }
- }
-
- /**
- * device id reference, for reuse device id in both id table and memtable <br>
- * used in memtable
- */
- public IDeviceID getDeviceID() {
- return deviceID;
- }
-
- public void setDeviceID(IDeviceID deviceID) {
- this.deviceID = deviceID;
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java
deleted file mode 100644
index 0e7bc1e7fe..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java
+++ /dev/null
@@ -1,812 +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.qp.physical.crud;
-
-import org.apache.iotdb.commons.conf.IoTDBConstant;
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.commons.utils.TestOnly;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.metadata.PathNotExistException;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
-import org.apache.iotdb.db.qp.logical.Operator;
-import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
-import org.apache.iotdb.db.utils.CommonUtils;
-import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
-import org.apache.iotdb.db.wal.buffer.WALEntryValue;
-import org.apache.iotdb.db.wal.utils.WALWriteUtils;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.tsfile.read.TimeValuePair;
-import org.apache.iotdb.tsfile.utils.Binary;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
-import org.apache.iotdb.tsfile.write.record.TSRecord;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Objects;
-
-public class InsertRowPlan extends InsertPlan implements WALEntryValue {
- private static final Logger logger = LoggerFactory.getLogger(InsertRowPlan.class);
- private static final byte TYPE_RAW_STRING = -1;
-
- private long time;
- private Object[] values;
-
- // if isNeedInferType is true, the values must be String[], so we could infer types from them
- // if values is object[], we could use the raw type of them, and we should set this to false
- private boolean isNeedInferType = false;
-
- private List<Object> failedValues;
- private List<PartialPath> paths;
-
- public InsertRowPlan() {
- super(OperatorType.INSERT);
- }
-
- public InsertRowPlan(InsertRowPlan another) {
- super(OperatorType.INSERT);
- this.devicePath = another.devicePath;
- this.time = another.time;
- this.measurements = new String[another.measurements.length];
- System.arraycopy(another.measurements, 0, this.measurements, 0, another.measurements.length);
- this.values = new Object[another.values.length];
- System.arraycopy(another.values, 0, this.values, 0, another.values.length);
- this.dataTypes = new TSDataType[another.dataTypes.length];
- System.arraycopy(another.dataTypes, 0, this.dataTypes, 0, another.dataTypes.length);
- }
-
- public InsertRowPlan(
- PartialPath prefixPath, long insertTime, String[] measurementList, String[] insertValues) {
- super(Operator.OperatorType.INSERT);
- this.time = insertTime;
- this.devicePath = prefixPath;
- this.measurements = measurementList;
- this.dataTypes = new TSDataType[insertValues.length];
- // We need to create an Object[] for the data type casting, because we can not set Float, Long
- // to String[i]
- this.values = new Object[insertValues.length];
- System.arraycopy(insertValues, 0, values, 0, insertValues.length);
- isNeedInferType = true;
- }
-
- public InsertRowPlan(
- PartialPath prefixPath,
- long insertTime,
- String[] measurementList,
- ByteBuffer values,
- boolean isAligned)
- throws QueryProcessException {
- super(Operator.OperatorType.INSERT);
- this.time = insertTime;
- this.devicePath = prefixPath;
- this.measurements = measurementList;
- this.dataTypes = new TSDataType[measurementList.length];
- this.values = new Object[measurementList.length];
- this.fillValues(values);
- isNeedInferType = false;
- this.isAligned = isAligned;
- }
-
- @TestOnly
- public InsertRowPlan(
- PartialPath prefixPath,
- long insertTime,
- String[] measurements,
- TSDataType[] dataTypes,
- String[] insertValues) {
- super(OperatorType.INSERT);
- this.time = insertTime;
- this.devicePath = prefixPath;
- this.measurements = measurements;
- this.dataTypes = dataTypes;
- this.values = new Object[dataTypes.length];
- for (int i = 0; i < dataTypes.length; i++) {
- try {
- values[i] = CommonUtils.parseValueForTest(dataTypes[i], insertValues[i]);
- } catch (QueryProcessException e) {
- e.printStackTrace();
- }
- }
- }
-
- @TestOnly
- public InsertRowPlan(
- PartialPath prefixPath,
- long insertTime,
- String[] measurements,
- TSDataType[] dataTypes,
- String[] insertValues,
- boolean isAligned) {
- super(OperatorType.INSERT);
- this.time = insertTime;
- this.devicePath = prefixPath;
- this.measurements = measurements;
- this.dataTypes = dataTypes;
- this.values = new Object[dataTypes.length];
- for (int i = 0; i < dataTypes.length; i++) {
- try {
- values[i] = CommonUtils.parseValueForTest(dataTypes[i], insertValues[i]);
- } catch (QueryProcessException e) {
- e.printStackTrace();
- }
- }
- this.isAligned = isAligned;
- }
-
- @TestOnly
- public InsertRowPlan(
- PartialPath prefixPath,
- long insertTime,
- String measurement,
- TSDataType type,
- String insertValue) {
- super(OperatorType.INSERT);
- this.time = insertTime;
- this.devicePath = prefixPath;
- this.measurements = new String[] {measurement};
- this.dataTypes = new TSDataType[] {type};
- this.values = new Object[1];
- try {
- values[0] = CommonUtils.parseValueForTest(dataTypes[0], insertValue);
- } catch (QueryProcessException e) {
- logger.error(e.getMessage());
- }
- }
-
- @TestOnly
- public InsertRowPlan(TSRecord tsRecord) throws IllegalPathException {
- super(OperatorType.INSERT);
- this.devicePath = new PartialPath(tsRecord.deviceId);
- this.time = tsRecord.time;
- this.measurements = new String[tsRecord.dataPointList.size()];
- this.measurementMNodes = new IMeasurementMNode[tsRecord.dataPointList.size()];
- this.dataTypes = new TSDataType[tsRecord.dataPointList.size()];
- this.values = new Object[tsRecord.dataPointList.size()];
- for (int i = 0; i < tsRecord.dataPointList.size(); i++) {
- measurements[i] = tsRecord.dataPointList.get(i).getMeasurementId();
- measurementMNodes[i] =
- MeasurementMNode.getMeasurementMNode(
- null,
- measurements[i],
- new MeasurementSchema(
- measurements[i], tsRecord.dataPointList.get(i).getType(), TSEncoding.PLAIN),
- null);
- dataTypes[i] = tsRecord.dataPointList.get(i).getType();
- values[i] = tsRecord.dataPointList.get(i).getValue();
- }
- }
-
- public long getTime() {
- return time;
- }
-
- public void setTime(long time) {
- this.time = time;
- }
-
- public boolean isNeedInferType() {
- return isNeedInferType;
- }
-
- public void setNeedInferType(boolean inferType) {
- this.isNeedInferType = inferType;
- }
-
- /**
- * if inferType is true, transfer String[] values to specific data types (Integer, Long, Float,
- * Double, Binary)
- */
- @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
- public void transferType() throws QueryProcessException {
- if (isNeedInferType) {
- for (int i = 0; i < measurementMNodes.length; i++) {
- if (measurementMNodes[i] == null) {
- if (IoTDBDescriptor.getInstance().getConfig().isEnablePartialInsert()) {
- markFailedMeasurementInsertion(
- i,
- new QueryProcessException(
- new PathNotExistException(
- devicePath.getFullPath()
- + IoTDBConstant.PATH_SEPARATOR
- + measurements[i])));
- } else {
- throw new QueryProcessException(
- new PathNotExistException(
- devicePath.getFullPath() + IoTDBConstant.PATH_SEPARATOR + measurements[i]));
- }
- continue;
- }
-
- dataTypes[i] = measurementMNodes[i].getSchema().getType();
- try {
- values[i] = CommonUtils.parseValue(dataTypes[i], values[i].toString());
- } catch (Exception e) {
- logger.warn(
- "data type of {}.{} is not consistent, registered type {}, inserting timestamp {}, value {}",
- devicePath,
- measurements[i],
- dataTypes[i],
- time,
- values[i]);
- if (IoTDBDescriptor.getInstance().getConfig().isEnablePartialInsert()) {
- markFailedMeasurementInsertion(i, e);
- measurementMNodes[i] = null;
- } else {
- throw e;
- }
- }
- }
- }
- }
-
- @Override
- public long getMinTime() {
- return getTime();
- }
-
- @Override
- public Object getFirstValueOfIndex(int index) {
- return values[index];
- }
-
- @Override
- public void markFailedMeasurementInsertion(int index, Exception e) {
- if (measurements[index] == null) {
- return;
- }
- super.markFailedMeasurementInsertion(index, e);
- if (failedValues == null) {
- failedValues = new ArrayList<>();
- }
- failedValues.add(values[index]);
- values[index] = null;
- if (isNeedInferType) {
- dataTypes[index] = null;
- }
- }
-
- @Override
- public List<PartialPath> getPaths() {
- if (paths != null) {
- return paths;
- }
- paths = new ArrayList<>(measurements.length);
- for (String m : measurements) {
- PartialPath fullPath = devicePath.concatNode(m);
- paths.add(fullPath);
- }
- return paths;
- }
-
- public Object[] getValues() {
- return this.values;
- }
-
- public void setValues(Object[] values) {
- this.values = values;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- InsertRowPlan that = (InsertRowPlan) o;
- return time == that.time
- && Objects.equals(devicePath, that.devicePath)
- && Arrays.equals(measurements, that.measurements)
- && Arrays.equals(values, that.values)
- && Objects.equals(isAligned, that.isAligned);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(devicePath, time);
- }
-
- @Override
- public int serializedSize() {
- int size = 0;
- size += Byte.BYTES;
- return size + subSerializeSize();
- }
-
- int subSerializeSize() {
- int size = 0;
- size += Long.BYTES;
- size += ReadWriteIOUtils.sizeToWrite(devicePath.getFullPath());
- return size + serializeMeasurementsAndValuesSize();
- }
-
- int serializeMeasurementsAndValuesSize() {
- int size = 0;
- size += Integer.BYTES;
- for (String m : measurements) {
- if (m != null) {
- size += ReadWriteIOUtils.sizeToWrite(m);
- }
- }
- // putValues
- size += Integer.BYTES;
- for (int i = 0; i < values.length; i++) {
- if (values[i] == null) {
- continue;
- }
- if (dataTypes == null || dataTypes[i] == null) {
- size += Byte.BYTES + ReadWriteIOUtils.sizeToWrite(values[i].toString());
- } else {
- size += Byte.BYTES;
- switch (dataTypes[i]) {
- case BOOLEAN:
- size += Byte.BYTES;
- break;
- case INT32:
- size += Integer.BYTES;
- break;
- case INT64:
- size += Long.BYTES;
- break;
- case FLOAT:
- size += Float.BYTES;
- break;
- case DOUBLE:
- size += Double.BYTES;
- break;
- case TEXT:
- size += ReadWriteIOUtils.sizeToWrite((Binary) values[i]);
- break;
- }
- }
- }
- size += Byte.BYTES;
- size += Long.BYTES;
- size += Byte.BYTES;
- return size;
- }
-
- @Override
- public void serialize(DataOutputStream stream) throws IOException {
- int type = PhysicalPlanType.INSERT.ordinal();
- stream.writeByte((byte) type);
- subSerialize(stream);
- }
-
- public void subSerialize(DataOutputStream stream) throws IOException {
- stream.writeLong(time);
- putString(stream, devicePath.getFullPath());
- serializeMeasurementsAndValues(stream);
- }
-
- void serializeMeasurementsAndValues(DataOutputStream stream) throws IOException {
- stream.writeInt(measurements.length - getFailedMeasurementNumber());
-
- for (String m : measurements) {
- if (m != null) {
- putString(stream, m);
- }
- }
-
- try {
- stream.writeInt(values.length - getFailedMeasurementNumber());
- putValues(stream);
- } catch (QueryProcessException e) {
- throw new IOException(e);
- }
-
- // the types are not inferred before the plan is serialized
- stream.write((byte) (isNeedInferType ? 1 : 0));
-
- stream.writeLong(index);
- stream.write((byte) (isAligned ? 1 : 0));
- }
-
- private void putValues(DataOutputStream outputStream) throws QueryProcessException, IOException {
- for (int i = 0; i < values.length; i++) {
- if (values[i] == null) {
- continue;
- }
- // types are not determined, the situation mainly occurs when the plan uses string values
- // and is forwarded to other nodes
- if (dataTypes == null || dataTypes[i] == null) {
- ReadWriteIOUtils.write(TYPE_RAW_STRING, outputStream);
- ReadWriteIOUtils.write(values[i].toString(), outputStream);
- } else {
- ReadWriteIOUtils.write(dataTypes[i], outputStream);
- switch (dataTypes[i]) {
- case BOOLEAN:
- ReadWriteIOUtils.write((Boolean) values[i], outputStream);
- break;
- case INT32:
- ReadWriteIOUtils.write((Integer) values[i], outputStream);
- break;
- case INT64:
- ReadWriteIOUtils.write((Long) values[i], outputStream);
- break;
- case FLOAT:
- ReadWriteIOUtils.write((Float) values[i], outputStream);
- break;
- case DOUBLE:
- ReadWriteIOUtils.write((Double) values[i], outputStream);
- break;
- case TEXT:
- ReadWriteIOUtils.write((Binary) values[i], outputStream);
- break;
- default:
- throw new QueryProcessException("Unsupported data type:" + dataTypes[i]);
- }
- }
- }
- }
-
- public void fillValues(DataInputStream stream) throws IOException, QueryProcessException {
- for (int i = 0; i < dataTypes.length; i++) {
- // types are not determined, the situation mainly occurs when the plan uses string values
- // and is forwarded to other nodes
- byte typeNum = stream.readByte();
- if (typeNum == TYPE_RAW_STRING) {
- values[i] = ReadWriteIOUtils.readString(stream);
- continue;
- }
- dataTypes[i] = TSDataType.values()[typeNum];
- switch (dataTypes[i]) {
- case BOOLEAN:
- values[i] = ReadWriteIOUtils.readBool(stream);
- break;
- case INT32:
- values[i] = ReadWriteIOUtils.readInt(stream);
- break;
- case INT64:
- values[i] = ReadWriteIOUtils.readLong(stream);
- break;
- case FLOAT:
- values[i] = ReadWriteIOUtils.readFloat(stream);
- break;
- case DOUBLE:
- values[i] = ReadWriteIOUtils.readDouble(stream);
- break;
- case TEXT:
- values[i] = ReadWriteIOUtils.readBinary(stream);
- break;
- default:
- throw new QueryProcessException("Unsupported data type:" + dataTypes[i]);
- }
- }
- }
-
- /** Make sure the values is already inited before calling this */
- public void fillValues(ByteBuffer buffer) throws QueryProcessException {
- for (int i = 0; i < dataTypes.length; i++) {
- // types are not determined, the situation mainly occurs when the plan uses string values
- // and is forwarded to other nodes
- byte typeNum = (byte) ReadWriteIOUtils.read(buffer);
- if (typeNum == TYPE_RAW_STRING) {
- values[i] = ReadWriteIOUtils.readString(buffer);
- continue;
- }
- dataTypes[i] = TSDataType.values()[typeNum];
- switch (dataTypes[i]) {
- case BOOLEAN:
- values[i] = ReadWriteIOUtils.readBool(buffer);
- break;
- case INT32:
- values[i] = ReadWriteIOUtils.readInt(buffer);
- break;
- case INT64:
- values[i] = ReadWriteIOUtils.readLong(buffer);
- break;
- case FLOAT:
- values[i] = ReadWriteIOUtils.readFloat(buffer);
- break;
- case DOUBLE:
- values[i] = ReadWriteIOUtils.readDouble(buffer);
- break;
- case TEXT:
- values[i] = ReadWriteIOUtils.readBinary(buffer);
- break;
- default:
- throw new QueryProcessException("Unsupported data type:" + dataTypes[i]);
- }
- }
- }
-
- @Override
- public void serializeImpl(ByteBuffer buffer) {
- int type = PhysicalPlanType.INSERT.ordinal();
- buffer.put((byte) type);
- subSerialize(buffer);
- }
-
- void subSerialize(ByteBuffer buffer) {
- buffer.putLong(time);
- putString(buffer, devicePath.getFullPath());
- serializeMeasurementsAndValues(buffer);
- }
-
- void serializeMeasurementsAndValues(ByteBuffer buffer) {
- buffer.putInt(measurements.length - getFailedMeasurementNumber());
-
- for (String measurement : measurements) {
- if (measurement != null) {
- putString(buffer, measurement);
- }
- }
- try {
- buffer.putInt(values.length - getFailedMeasurementNumber());
- putValues(buffer);
- } catch (QueryProcessException e) {
- logger.error("Failed to serialize values for {}", this, e);
- }
-
- // the types are not inferred before the plan is serialized
- buffer.put((byte) (isNeedInferType ? 1 : 0));
- buffer.putLong(index);
-
- buffer.put((byte) (isAligned ? 1 : 0));
- }
-
- private void putValues(ByteBuffer buffer) throws QueryProcessException {
- for (int i = 0; i < values.length; i++) {
- if (values[i] == null) {
- continue;
- }
- // types are not determined, the situation mainly occurs when the plan uses string values
- // and is forwarded to other nodes
- if (dataTypes == null || dataTypes[i] == null) {
- ReadWriteIOUtils.write(TYPE_RAW_STRING, buffer);
- ReadWriteIOUtils.write(values[i].toString(), buffer);
- } else {
- ReadWriteIOUtils.write(dataTypes[i], buffer);
- switch (dataTypes[i]) {
- case BOOLEAN:
- ReadWriteIOUtils.write((Boolean) values[i], buffer);
- break;
- case INT32:
- ReadWriteIOUtils.write((Integer) values[i], buffer);
- break;
- case INT64:
- ReadWriteIOUtils.write((Long) values[i], buffer);
- break;
- case FLOAT:
- ReadWriteIOUtils.write((Float) values[i], buffer);
- break;
- case DOUBLE:
- ReadWriteIOUtils.write((Double) values[i], buffer);
- break;
- case TEXT:
- ReadWriteIOUtils.write((Binary) values[i], buffer);
- break;
- default:
- throw new QueryProcessException("Unsupported data type:" + dataTypes[i]);
- }
- }
- }
- }
-
- @Override
- public void serializeToWAL(IWALByteBufferView buffer) {
- int type = PhysicalPlanType.INSERT.ordinal();
- buffer.put((byte) type);
- subSerialize(buffer);
- }
-
- void subSerialize(IWALByteBufferView buffer) {
- buffer.putLong(time);
- WALWriteUtils.write(devicePath.getFullPath(), buffer);
- serializeMeasurementsAndValues(buffer);
- }
-
- void serializeMeasurementsAndValues(IWALByteBufferView buffer) {
- buffer.putInt(measurements.length - getFailedMeasurementNumber());
-
- for (String measurement : measurements) {
- if (measurement != null) {
- WALWriteUtils.write(measurement, buffer);
- }
- }
- try {
- buffer.putInt(values.length - getFailedMeasurementNumber());
- putValues(buffer);
- } catch (QueryProcessException e) {
- logger.error("Failed to serialize values for {}", this, e);
- }
-
- // the types are not inferred before the plan is serialized
- buffer.put((byte) (isNeedInferType ? 1 : 0));
- buffer.putLong(index);
-
- buffer.put((byte) (isAligned ? 1 : 0));
- }
-
- private void putValues(IWALByteBufferView buffer) throws QueryProcessException {
- for (int i = 0; i < values.length; i++) {
- if (values[i] == null) {
- continue;
- }
- // types are not determined, the situation mainly occurs when the plan uses string values
- // and is forwarded to other nodes
- if (dataTypes == null || dataTypes[i] == null) {
- WALWriteUtils.write(TYPE_RAW_STRING, buffer);
- WALWriteUtils.write(values[i].toString(), buffer);
- } else {
- WALWriteUtils.write(dataTypes[i], buffer);
- switch (dataTypes[i]) {
- case BOOLEAN:
- WALWriteUtils.write((Boolean) values[i], buffer);
- break;
- case INT32:
- WALWriteUtils.write((Integer) values[i], buffer);
- break;
- case INT64:
- WALWriteUtils.write((Long) values[i], buffer);
- break;
- case FLOAT:
- WALWriteUtils.write((Float) values[i], buffer);
- break;
- case DOUBLE:
- WALWriteUtils.write((Double) values[i], buffer);
- break;
- case TEXT:
- WALWriteUtils.write((Binary) values[i], buffer);
- break;
- default:
- throw new QueryProcessException("Unsupported data type:" + dataTypes[i]);
- }
- }
- }
- }
-
- @Override
- public void deserialize(DataInputStream stream) throws IOException, IllegalPathException {
- this.time = stream.readLong();
- this.devicePath = new PartialPath(ReadWriteIOUtils.readString(stream));
- deserializeMeasurementsAndValues(stream);
- }
-
- void deserializeMeasurementsAndValues(DataInputStream stream) throws IOException {
- int measurementSize = stream.readInt();
-
- this.measurements = new String[measurementSize];
- for (int i = 0; i < measurementSize; i++) {
- measurements[i] = ReadWriteIOUtils.readString(stream);
- }
-
- int dataTypeSize = stream.readInt();
- this.dataTypes = new TSDataType[dataTypeSize];
- this.values = new Object[dataTypeSize];
- try {
- fillValues(stream);
- } catch (QueryProcessException e) {
- e.printStackTrace();
- }
-
- isNeedInferType = stream.readByte() == 1;
- this.index = stream.readLong();
- isAligned = stream.readByte() == 1;
- }
-
- @Override
- public void deserialize(ByteBuffer buffer) throws IllegalPathException {
- this.time = buffer.getLong();
- this.devicePath = new PartialPath(readString(buffer));
- deserializeMeasurementsAndValues(buffer);
- }
-
- void deserializeMeasurementsAndValues(ByteBuffer buffer) {
- int measurementSize = buffer.getInt();
-
- this.measurements = new String[measurementSize];
- for (int i = 0; i < measurementSize; i++) {
- measurements[i] = readString(buffer);
- }
-
- int dataTypeSize = buffer.getInt();
- this.dataTypes = new TSDataType[dataTypeSize];
- this.values = new Object[dataTypeSize];
- try {
- fillValues(buffer);
- } catch (QueryProcessException e) {
- e.printStackTrace();
- }
-
- isNeedInferType = buffer.get() == 1;
- this.index = buffer.getLong();
- isAligned = buffer.get() == 1;
- }
-
- @Override
- public String toString() {
- return "prefixPath: "
- + devicePath
- + ", time: "
- + time
- + ", measurements: "
- + Arrays.toString(measurements)
- + ", values: "
- + Arrays.toString(values);
- }
-
- boolean hasFailedValues() {
- return failedValues != null && !failedValues.isEmpty();
- }
-
- public TimeValuePair composeTimeValuePair(int columnIndex) {
- if (columnIndex >= values.length) {
- return null;
- }
- Object value = values[columnIndex];
- return new TimeValuePair(time, TsPrimitiveType.getByType(dataTypes[columnIndex], value));
- }
-
- @Override
- public InsertPlan getPlanFromFailed() {
- if (super.getPlanFromFailed() == null) {
- return null;
- }
- values = failedValues.toArray(new Object[0]);
- failedValues = null;
- return this;
- }
-
- @Override
- public void recoverFromFailure() {
- if (failedMeasurements == null) {
- return;
- }
-
- for (int i = 0; i < failedMeasurements.size(); i++) {
- int index = failedIndices.get(i);
- values[index] = failedValues.get(i);
- }
- super.recoverFromFailure();
-
- failedValues = null;
- }
-
- @Override
- public void checkIntegrity() throws QueryProcessException {
- super.checkIntegrity();
- if (values == null) {
- throw new QueryProcessException("Values are null");
- }
- if (values.length == 0) {
- throw new QueryProcessException("The size of values is 0");
- }
- for (Object value : values) {
- if (value == null) {
- throw new QueryProcessException("Values contain null: " + Arrays.toString(values));
- }
- }
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowsOfOneDevicePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowsOfOneDevicePlan.java
deleted file mode 100644
index 09bb299f54..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowsOfOneDevicePlan.java
+++ /dev/null
@@ -1,296 +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.qp.physical.crud;
-
-import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
-import org.apache.iotdb.db.qp.physical.BatchPlan;
-import org.apache.iotdb.tsfile.exception.NotImplementedException;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-public class InsertRowsOfOneDevicePlan extends InsertPlan implements BatchPlan {
-
- /**
- * This class has some duplicated codes with InsertRowsPlan, they should be refined in the future
- */
- boolean[] isExecuted;
-
- private InsertRowPlan[] rowPlans;
-
- /**
- * Suppose there is an InsertRowsOfOneDevicePlan, which contains 5 InsertRowPlans,
- * rowPlans={InsertRowPlan_0, InsertRowPlan_1, InsertRowPlan_2, InsertRowPlan_3, InsertRowPlan_4},
- * then the rowPlanIndexList={0, 1, 2, 3, 4} respectively. But when the InsertRowsOfOneDevicePlan
- * is split into two InsertRowsOfOneDevicePlans according to the time partition in cluster
- * version, suppose that the InsertRowsOfOneDevicePlan_1's rowPlanIndexList = {InsertRowPlan_0,
- * InsertRowPlan_3, InsertRowPlan_4}, then InsertRowsOfOneDevicePlan_1's rowPlanIndexList = {0, 3,
- * 4}; InsertRowsOfOneDevicePlan_2's rowPlanIndexList = {InsertRowPlan_1, InsertRowPlan_2} then
- * InsertRowsOfOneDevicePlan_2's rowPlanIndexList = {1, 2} respectively;
- */
- private int[] rowPlanIndexList;
-
- /** record the result of insert rows */
- private Map<Integer, TSStatus> results = new HashMap<>();
-
- private List<PartialPath> paths;
-
- public InsertRowsOfOneDevicePlan() {
- super(OperatorType.BATCH_INSERT_ONE_DEVICE);
- }
-
- public InsertRowsOfOneDevicePlan(
- PartialPath prefixPath,
- List<Long> insertTimes,
- List<List<String>> measurements,
- List<ByteBuffer> insertValues,
- boolean isAligned)
- throws QueryProcessException {
- this();
- this.devicePath = prefixPath;
- rowPlans = new InsertRowPlan[insertTimes.size()];
- rowPlanIndexList = new int[insertTimes.size()];
- for (int i = 0; i < insertTimes.size(); i++) {
- rowPlans[i] =
- new InsertRowPlan(
- prefixPath,
- insertTimes.get(i),
- measurements.get(i).toArray(new String[0]),
- insertValues.get(i),
- isAligned);
- if (rowPlans[i].getMeasurements().length == 0) {
- throw new QueryProcessException(
- "The measurements are null, deviceId:" + prefixPath + ", time:" + insertTimes.get(i));
- }
- if (rowPlans[i].getValues().length == 0) {
- throw new QueryProcessException(
- "The size of values in InsertRowsOfOneDevicePlan is 0, deviceId:"
- + prefixPath
- + ", time:"
- + insertTimes.get(i));
- }
- rowPlanIndexList[i] = i;
- }
- }
-
- /**
- * This constructor is used for splitting parent InsertRowsOfOneDevicePlan into sub ones. So
- * there's no need to validate rowPlans.
- */
- public InsertRowsOfOneDevicePlan(
- PartialPath prefixPath, InsertRowPlan[] rowPlans, int[] rowPlanIndexList) {
- this();
- this.devicePath = prefixPath;
- this.rowPlans = rowPlans;
- this.rowPlanIndexList = rowPlanIndexList;
- }
-
- @Override
- public void checkIntegrity() {}
-
- // TODO I see InsertRowPlan rewrites the hashCode, but do we need to rewrite hashCode?
-
- @Override
- public List<PartialPath> getPaths() {
- if (paths != null) {
- return paths;
- }
- Set<PartialPath> pathSet = new HashSet<>();
- for (InsertRowPlan plan : rowPlans) {
- pathSet.addAll(plan.getPaths());
- }
- paths = new ArrayList<>(pathSet);
- return paths;
- }
-
- @Override
- public long getMinTime() {
- long minTime = Long.MAX_VALUE;
- for (InsertRowPlan plan : rowPlans) {
- if (minTime > plan.getTime()) {
- minTime = plan.getTime();
- }
- }
- return minTime;
- }
-
- @Override
- public Object getFirstValueOfIndex(int index) {
- throw new NotImplementedException();
- }
-
- @Override
- public void serialize(DataOutputStream stream) throws IOException {
- int type = PhysicalPlanType.BATCH_INSERT_ONE_DEVICE.ordinal();
- stream.writeByte((byte) type);
- putString(stream, devicePath.getFullPath());
-
- stream.writeInt(rowPlans.length);
- for (InsertRowPlan plan : rowPlans) {
- stream.writeLong(plan.getTime());
- plan.serializeMeasurementsAndValues(stream);
- }
- for (Integer index : rowPlanIndexList) {
- stream.writeInt(index);
- }
- }
-
- @Override
- public void serializeImpl(ByteBuffer buffer) {
- int type = PhysicalPlanType.BATCH_INSERT_ONE_DEVICE.ordinal();
- buffer.put((byte) type);
-
- putString(buffer, devicePath.getFullPath());
- buffer.putInt(rowPlans.length);
- for (InsertRowPlan plan : rowPlans) {
- buffer.putLong(plan.getTime());
- plan.serializeMeasurementsAndValues(buffer);
- }
- for (Integer index : rowPlanIndexList) {
- buffer.putInt(index);
- }
- }
-
- @Override
- public void deserialize(ByteBuffer buffer) throws IllegalPathException {
- this.devicePath = new PartialPath(readString(buffer));
- this.rowPlans = new InsertRowPlan[buffer.getInt()];
- for (int i = 0; i < rowPlans.length; i++) {
- rowPlans[i] = new InsertRowPlan();
- rowPlans[i].setDevicePath(devicePath);
- rowPlans[i].setTime(buffer.getLong());
- rowPlans[i].deserializeMeasurementsAndValues(buffer);
- }
- this.rowPlanIndexList = new int[rowPlans.length];
- for (int i = 0; i < rowPlans.length; i++) {
- rowPlanIndexList[i] = buffer.getInt();
- }
- }
-
- @Override
- public void setIndex(long index) {
- super.setIndex(index);
- for (InsertRowPlan plan : rowPlans) {
- // use the InsertRowsOfOneDevicePlan's index as the sub InsertRowPlan's index
- plan.setIndex(index);
- }
- }
-
- @Override
- public String toString() {
- return "deviceId: " + devicePath + ", times: " + rowPlans.length;
- }
-
- @Override
- public InsertPlan getPlanFromFailed() {
- if (super.getPlanFromFailed() == null) {
- return null;
- }
- List<InsertRowPlan> plans = new ArrayList<>();
- for (InsertRowPlan plan : rowPlans) {
- if (plan.hasFailedValues()) {
- plans.add((InsertRowPlan) plan.getPlanFromFailed());
- }
- }
- this.rowPlans = plans.toArray(new InsertRowPlan[0]);
- return this;
- }
-
- public InsertRowPlan[] getRowPlans() {
- return rowPlans;
- }
-
- @Override
- public void setIsExecuted(int i) {
- if (isExecuted == null) {
- isExecuted = new boolean[getBatchSize()];
- }
- isExecuted[i] = true;
- }
-
- @Override
- public boolean isExecuted(int i) {
- if (isExecuted == null) {
- isExecuted = new boolean[getBatchSize()];
- }
- return isExecuted[i];
- }
-
- @Override
- public Map<Integer, TSStatus> getResults() {
- return results;
- }
-
- @Override
- public List<PartialPath> getPrefixPaths() {
- return Collections.singletonList(this.devicePath);
- }
-
- @Override
- public int getBatchSize() {
- return rowPlans.length;
- }
-
- public int[] getRowPlanIndexList() {
- return rowPlanIndexList;
- }
-
- @Override
- public void unsetIsExecuted(int i) {
- if (isExecuted == null) {
- isExecuted = new boolean[getBatchSize()];
- }
- isExecuted[i] = false;
- if (rowPlanIndexList != null && rowPlanIndexList.length > 0) {
- results.remove(rowPlanIndexList[i]);
- } else {
- results.remove(i);
- }
- }
-
- @Override
- public boolean equals(Object o) {
- return o instanceof InsertRowsOfOneDevicePlan
- && Arrays.equals(((InsertRowsOfOneDevicePlan) o).rowPlanIndexList, this.rowPlanIndexList)
- && Arrays.equals(((InsertRowsOfOneDevicePlan) o).rowPlans, this.rowPlans)
- && ((InsertRowsOfOneDevicePlan) o).results.equals(this.results)
- && ((InsertRowsOfOneDevicePlan) o).getDevicePath().equals(this.getDevicePath());
- }
-
- @Override
- public int hashCode() {
- int result = rowPlans != null ? Arrays.hashCode(rowPlans) : 0;
- result = 31 * result + (rowPlanIndexList != null ? Arrays.hashCode(rowPlanIndexList) : 0);
- result = 31 * result + (results != null ? results.hashCode() : 0);
- return result;
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowsPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowsPlan.java
deleted file mode 100644
index 9faf111599..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowsPlan.java
+++ /dev/null
@@ -1,302 +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.qp.physical.crud;
-
-import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.commons.utils.StatusUtils;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
-import org.apache.iotdb.db.qp.physical.BatchPlan;
-import org.apache.iotdb.tsfile.exception.NotImplementedException;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-
-public class InsertRowsPlan extends InsertPlan implements BatchPlan {
-
- /**
- * Suppose there is an InsertRowsPlan, which contains 5 InsertRowPlans,
- * insertRowPlanList={InsertRowPlan_0, InsertRowPlan_1, InsertRowPlan_2, InsertRowPlan_3,
- * InsertRowPlan_4}, then the insertRowPlanIndexList={0, 1, 2, 3, 4} respectively. But when the
- * InsertRowsPlan is split into two InsertRowsPlans according to different storage group in
- * cluster version, suppose that the InsertRowsPlan_1's insertRowPlanList = {InsertRowPlan_0,
- * InsertRowPlan_3, InsertRowPlan_4}, then InsertRowsPlan_1's insertRowPlanIndexList = {0, 3, 4};
- * InsertRowsPlan_2's insertRowPlanList = {InsertRowPlan_1, * InsertRowPlan_2} then
- * InsertRowsPlan_2's insertRowPlanIndexList= {1, 2} respectively;
- */
- private List<Integer> insertRowPlanIndexList;
-
- /** the InsertRowsPlan list */
- private List<InsertRowPlan> insertRowPlanList;
-
- boolean[] isExecuted;
-
- /** record the result of insert rows */
- private Map<Integer, TSStatus> results = new HashMap<>();
-
- private List<PartialPath> paths;
- private List<PartialPath> prefixPaths;
-
- public InsertRowsPlan() {
- super(OperatorType.BATCH_INSERT_ROWS);
- insertRowPlanList = new ArrayList<>();
- insertRowPlanIndexList = new ArrayList<>();
- }
-
- @Override
- public long getMinTime() {
- long minTime = Long.MAX_VALUE;
- for (InsertRowPlan insertRowPlan : insertRowPlanList) {
- if (insertRowPlan.getMinTime() < minTime) {
- minTime = insertRowPlan.getMinTime();
- }
- }
- return minTime;
- }
-
- @Override
- public Object getFirstValueOfIndex(int index) {
- throw new NotImplementedException();
- }
-
- @Override
- public List<PartialPath> getPaths() {
- if (paths != null) {
- return paths;
- }
- Set<PartialPath> pathSet = new HashSet<>();
- for (InsertRowPlan plan : insertRowPlanList) {
- pathSet.addAll(plan.getPaths());
- }
- paths = new ArrayList<>(pathSet);
- return paths;
- }
-
- @Override
- public List<PartialPath> getPrefixPaths() {
- if (prefixPaths != null) {
- return prefixPaths;
- }
- prefixPaths = new ArrayList<>(insertRowPlanList.size());
- for (InsertRowPlan insertRowPlan : insertRowPlanList) {
- prefixPaths.add(insertRowPlan.getDevicePath());
- }
- return prefixPaths;
- }
-
- @Override
- public void checkIntegrity() throws QueryProcessException {
- if (insertRowPlanList.isEmpty()) {
- throw new QueryProcessException("sub plan are empty.");
- }
- for (InsertRowPlan insertRowPlan : insertRowPlanList) {
- insertRowPlan.checkIntegrity();
- }
- }
-
- @Override
- public void recoverFromFailure() {
- for (InsertRowPlan insertRowPlan : insertRowPlanList) {
- insertRowPlan.recoverFromFailure();
- }
- }
-
- @Override
- public InsertPlan getPlanFromFailed() {
- if (super.getPlanFromFailed() == null) {
- return null;
- }
- List<InsertRowPlan> plans = new ArrayList<>();
- List<Integer> indexes = new ArrayList<>();
- for (int i = 0; i < insertRowPlanList.size(); i++) {
- if (insertRowPlanList.get(i).hasFailedValues()) {
- plans.add((InsertRowPlan) insertRowPlanList.get(i).getPlanFromFailed());
- indexes.add(i);
- }
- }
- this.insertRowPlanList = plans;
- this.insertRowPlanIndexList = indexes;
- return this;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- InsertRowsPlan that = (InsertRowsPlan) o;
-
- if (!Objects.equals(insertRowPlanIndexList, that.insertRowPlanIndexList)) {
- return false;
- }
- if (!Objects.equals(insertRowPlanList, that.insertRowPlanList)) {
- return false;
- }
- return Objects.equals(results, that.results);
- }
-
- @Override
- public int hashCode() {
- int result = insertRowPlanIndexList != null ? insertRowPlanIndexList.hashCode() : 0;
- result = 31 * result + (insertRowPlanList != null ? insertRowPlanList.hashCode() : 0);
- result = 31 * result + (results != null ? results.hashCode() : 0);
- return result;
- }
-
- @Override
- public void serializeImpl(ByteBuffer buffer) {
- int type = PhysicalPlanType.BATCH_INSERT_ROWS.ordinal();
- buffer.put((byte) type);
- buffer.putInt(insertRowPlanList.size());
- for (InsertRowPlan insertRowPlan : insertRowPlanList) {
- insertRowPlan.subSerialize(buffer);
- }
- for (Integer index : insertRowPlanIndexList) {
- buffer.putInt(index);
- }
- }
-
- @Override
- public void serialize(DataOutputStream stream) throws IOException {
- int type = PhysicalPlanType.BATCH_INSERT_ROWS.ordinal();
- stream.writeByte((byte) type);
- stream.writeInt(insertRowPlanList.size());
- for (InsertRowPlan insertRowPlan : insertRowPlanList) {
- insertRowPlan.subSerialize(stream);
- }
- for (Integer index : insertRowPlanIndexList) {
- stream.writeInt(index);
- }
- }
-
- @Override
- public void deserialize(ByteBuffer buffer) throws IllegalPathException {
- int size = buffer.getInt();
- this.insertRowPlanList = new ArrayList<>(size);
- this.insertRowPlanIndexList = new ArrayList<>(size);
- for (int i = 0; i < size; i++) {
- InsertRowPlan insertRowPlan = new InsertRowPlan();
- insertRowPlan.deserialize(buffer);
- insertRowPlanList.add(insertRowPlan);
- }
-
- for (int i = 0; i < size; i++) {
- insertRowPlanIndexList.add(buffer.getInt());
- }
- }
-
- @Override
- public void setIndex(long index) {
- super.setIndex(index);
- for (InsertRowPlan insertRowPlan : insertRowPlanList) {
- // use the InsertRowsPlan's index as the sub InsertRowPlan's index
- insertRowPlan.setIndex(index);
- }
- }
-
- @Override
- public Map<Integer, TSStatus> getResults() {
- return results;
- }
-
- public void addOneInsertRowPlan(InsertRowPlan plan, int index) {
- insertRowPlanList.add(plan);
- insertRowPlanIndexList.add(index);
- }
-
- public List<Integer> getInsertRowPlanIndexList() {
- return insertRowPlanIndexList;
- }
-
- public List<InsertRowPlan> getInsertRowPlanList() {
- return insertRowPlanList;
- }
-
- public int getRowCount() {
- return insertRowPlanList.size();
- }
-
- public PartialPath getFirstDeviceId() {
- return insertRowPlanList.get(0).getDevicePath();
- }
-
- @Override
- public String toString() {
- return "InsertRowsPlan{"
- + " insertRowPlanIndexList's size="
- + insertRowPlanIndexList.size()
- + ", insertRowPlanList's size="
- + insertRowPlanList.size()
- + ", results="
- + results
- + "}";
- }
-
- public TSStatus[] getFailingStatus() {
- return StatusUtils.getFailingStatus(results, insertRowPlanList.size());
- }
-
- @Override
- public void setIsExecuted(int i) {
- if (isExecuted == null) {
- isExecuted = new boolean[getBatchSize()];
- }
- isExecuted[i] = true;
- }
-
- @Override
- public boolean isExecuted(int i) {
- if (isExecuted == null) {
- isExecuted = new boolean[getBatchSize()];
- }
- return isExecuted[i];
- }
-
- @Override
- public int getBatchSize() {
- return insertRowPlanList.size();
- }
-
- @Override
- public void unsetIsExecuted(int i) {
- if (isExecuted == null) {
- isExecuted = new boolean[getBatchSize()];
- }
- isExecuted[i] = false;
- if (insertRowPlanIndexList != null && !insertRowPlanIndexList.isEmpty()) {
- results.remove(insertRowPlanIndexList.get(i));
- } else {
- results.remove(i);
- }
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertTabletPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertTabletPlan.java
deleted file mode 100644
index 57e9995c66..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertTabletPlan.java
+++ /dev/null
@@ -1,895 +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.qp.physical.crud;
-
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
-import org.apache.iotdb.db.utils.QueryDataSetUtils;
-import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
-import org.apache.iotdb.db.wal.buffer.WALEntryValue;
-import org.apache.iotdb.db.wal.utils.WALWriteUtils;
-import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.read.TimeValuePair;
-import org.apache.iotdb.tsfile.utils.Binary;
-import org.apache.iotdb.tsfile.utils.BitMap;
-import org.apache.iotdb.tsfile.utils.BytesUtils;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsBinary;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsBoolean;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsDouble;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsFloat;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsInt;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsLong;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Objects;
-
-@SuppressWarnings("java:S1135") // ignore todos
-public class InsertTabletPlan extends InsertPlan implements WALEntryValue {
- private static final String DATATYPE_UNSUPPORTED = "Data type %s is not supported.";
-
- private long[] times; // times should be sorted. It is done in the session API.
-
- private BitMap[] bitMaps;
- private Object[] columns;
- private int rowCount = 0;
- private List<PartialPath> paths;
- // when this plan is sub-plan split from another InsertTabletPlan, this indicates the original
- // positions of values in
- // this plan. For example, if the plan contains 5 timestamps, and range = [1,4,10,12], then it
- // means that the first 3
- // timestamps in this plan are from range[1,4) of the parent plan, and the last 2 timestamps are
- // from range[10,12)
- // of the parent plan.
- // this is usually used to back-propagate exceptions to the parent plan without losing their
- // proper positions.
- private List<Integer> range;
-
- private List<Object> failedColumns;
-
- public InsertTabletPlan() {
- super(OperatorType.BATCH_INSERT);
- }
-
- public InsertTabletPlan(PartialPath prefixPath, List<String> measurements) {
- super(OperatorType.BATCH_INSERT);
- this.devicePath = prefixPath;
- this.measurements = measurements.toArray(new String[0]);
- this.canBeSplit = true;
- }
-
- public InsertTabletPlan(PartialPath prefixPath, String[] measurements) {
- super(OperatorType.BATCH_INSERT);
- this.devicePath = prefixPath;
- this.measurements = measurements;
- this.canBeSplit = true;
- }
-
- public InsertTabletPlan(PartialPath prefixPath, String[] measurements, List<Integer> dataTypes) {
- super(OperatorType.BATCH_INSERT);
- this.devicePath = prefixPath;
- this.measurements = measurements;
- setDataTypes(dataTypes);
- this.canBeSplit = true;
- }
-
- public InsertTabletPlan(
- PartialPath prefixPath, String[] measurements, List<Integer> dataTypes, boolean isAligned) {
- super(OperatorType.BATCH_INSERT);
- this.devicePath = prefixPath;
- this.measurements = measurements;
- setDataTypes(dataTypes);
- this.canBeSplit = true;
- this.isAligned = isAligned;
- }
-
- public List<Integer> getRange() {
- return range;
- }
-
- public void setRange(List<Integer> range) {
- this.range = range;
- }
-
- @Override
- public List<PartialPath> getPaths() {
- if (paths != null) {
- return paths;
- }
- List<PartialPath> ret = new ArrayList<>();
- for (String m : measurements) {
- PartialPath fullPath = devicePath.concatNode(m);
- ret.add(fullPath);
- }
- paths = ret;
- return ret;
- }
-
- @Override
- public int serializedSize() {
- int size = Byte.BYTES;
- return size + subSerializeSize(0, rowCount);
- }
-
- public int serializedSize(int start, int end) {
- int size = Byte.BYTES;
- return size + subSerializeSize(start, end);
- }
-
- int subSerializeSize(int start, int end) {
- int size = 0;
- size += ReadWriteIOUtils.sizeToWrite(devicePath.getFullPath());
- // measurements size
- size += Integer.BYTES;
- for (String m : measurements) {
- if (m != null) {
- size += ReadWriteIOUtils.sizeToWrite(m);
- }
- }
- // data types size
- size += Integer.BYTES;
- for (int i = 0; i < dataTypes.length; i++) {
- if (columns[i] != null) {
- size += Byte.BYTES;
- }
- }
- // times size
- size += Integer.BYTES;
- size += Long.BYTES * (end - start);
- // bitmaps size
- size += Byte.BYTES;
- if (bitMaps != null) {
- for (int i = 0; i < bitMaps.length; ++i) {
- if (columns[i] == null) {
- continue;
- }
-
- size += Byte.BYTES;
- if (bitMaps[i] != null) {
- int len = end - start;
- BitMap partBitMap = new BitMap(len);
- BitMap.copyOfRange(bitMaps[i], start, partBitMap, 0, len);
- size += partBitMap.getByteArray().length;
- }
- }
- }
- // values size
- for (int i = 0; i < dataTypes.length; i++) {
- if (columns[i] != null) {
- size += getColumnSize(dataTypes[i], columns[i], start, end);
- }
- }
- size += Long.BYTES;
- size += Byte.BYTES;
- return size;
- }
-
- private int getColumnSize(TSDataType dataType, Object column, int start, int end) {
- int size = 0;
- switch (dataType) {
- case INT32:
- size += Integer.BYTES * (end - start);
- break;
- case INT64:
- size += Long.BYTES * (end - start);
- break;
- case FLOAT:
- size += Float.BYTES * (end - start);
- break;
- case DOUBLE:
- size += Double.BYTES * (end - start);
- break;
- case BOOLEAN:
- size += Byte.BYTES * (end - start);
- break;
- case TEXT:
- Binary[] binaryValues = (Binary[]) column;
- for (int j = start; j < end; j++) {
- size += ReadWriteIOUtils.sizeToWrite(binaryValues[j]);
- }
- break;
- }
- return size;
- }
-
- @Override
- public void serialize(DataOutputStream stream) throws IOException {
- int type = PhysicalPlanType.BATCHINSERT.ordinal();
- stream.writeByte((byte) type);
- subSerialize(stream, 0, rowCount);
- }
-
- void subSerialize(DataOutputStream stream, int start, int end) throws IOException {
- putString(stream, devicePath.getFullPath());
- writeMeasurements(stream);
- writeDataTypes(stream);
- writeTimes(stream, start, end);
- writeBitMaps(stream, start, end);
- writeValues(stream, start, end);
- stream.write((byte) (isAligned ? 1 : 0));
- }
-
- private void writeMeasurements(DataOutputStream stream) throws IOException {
- stream.writeInt(measurements.length - getFailedMeasurementNumber());
- for (String m : measurements) {
- if (m == null) {
- continue;
- }
- putString(stream, m);
- }
- }
-
- private void writeDataTypes(DataOutputStream stream) throws IOException {
- stream.writeInt(dataTypes.length - getFailedMeasurementNumber());
- for (int i = 0; i < dataTypes.length; i++) {
- if (columns[i] == null) {
- continue;
- }
- dataTypes[i].serializeTo(stream);
- }
- }
-
- private void writeTimes(DataOutputStream stream, int start, int end) throws IOException {
- stream.writeInt(end - start);
- for (int i = start; i < end; i++) {
- stream.writeLong(times[i]);
- }
- }
-
- private void writeBitMaps(DataOutputStream stream, int start, int end) throws IOException {
- stream.writeBoolean(bitMaps != null);
- if (bitMaps != null) {
- for (int i = 0; i < bitMaps.length; ++i) {
- if (columns[i] == null) {
- continue;
- }
-
- if (bitMaps[i] == null) {
- stream.writeBoolean(false);
- } else {
- stream.writeBoolean(true);
- int len = end - start;
- BitMap partBitMap = new BitMap(len);
- BitMap.copyOfRange(bitMaps[i], start, partBitMap, 0, len);
- stream.write(partBitMap.getByteArray());
- }
- }
- }
- }
-
- private void writeValues(DataOutputStream stream, int start, int end) throws IOException {
- serializeValues(stream, start, end);
- stream.writeLong(index);
- }
-
- private void serializeValues(DataOutputStream outputStream, int start, int end)
- throws IOException {
- for (int i = 0; i < columns.length; i++) {
- if (columns[i] == null) {
- continue;
- }
- serializeColumn(dataTypes[i], columns[i], outputStream, start, end);
- }
- }
-
- private void serializeColumn(
- TSDataType dataType, Object column, DataOutputStream outputStream, int start, int end)
- throws IOException {
- switch (dataType) {
- case INT32:
- int[] intValues = (int[]) column;
- for (int j = start; j < end; j++) {
- outputStream.writeInt(intValues[j]);
- }
- break;
- case INT64:
- long[] longValues = (long[]) column;
- for (int j = start; j < end; j++) {
- outputStream.writeLong(longValues[j]);
- }
- break;
- case FLOAT:
- float[] floatValues = (float[]) column;
- for (int j = start; j < end; j++) {
- outputStream.writeFloat(floatValues[j]);
- }
- break;
- case DOUBLE:
- double[] doubleValues = (double[]) column;
- for (int j = start; j < end; j++) {
- outputStream.writeDouble(doubleValues[j]);
- }
- break;
- case BOOLEAN:
- boolean[] boolValues = (boolean[]) column;
- for (int j = start; j < end; j++) {
- outputStream.writeByte(BytesUtils.boolToByte(boolValues[j]));
- }
- break;
- case TEXT:
- Binary[] binaryValues = (Binary[]) column;
- for (int j = start; j < end; j++) {
- outputStream.writeInt(binaryValues[j].getLength());
- outputStream.write(binaryValues[j].getValues());
- }
- break;
- default:
- throw new UnSupportedDataTypeException(String.format(DATATYPE_UNSUPPORTED, dataType));
- }
- }
-
- @Override
- public void serializeImpl(ByteBuffer buffer) {
- int type = PhysicalPlanType.BATCHINSERT.ordinal();
- buffer.put((byte) type);
- subSerialize(buffer, 0, rowCount);
- }
-
- void subSerialize(ByteBuffer buffer, int start, int end) {
- putString(buffer, devicePath.getFullPath());
- writeMeasurements(buffer);
- writeDataTypes(buffer);
- writeTimes(buffer, 0, rowCount);
- writeBitMaps(buffer, 0, rowCount);
- writeValues(buffer, 0, rowCount);
- buffer.put((byte) (isAligned ? 1 : 0));
- }
-
- private void writeMeasurements(ByteBuffer buffer) {
- buffer.putInt(measurements.length - getFailedMeasurementNumber());
- for (String m : measurements) {
- if (m != null) {
- putString(buffer, m);
- }
- }
- }
-
- private void writeDataTypes(ByteBuffer buffer) {
- buffer.putInt(dataTypes.length - getFailedMeasurementNumber());
- for (int i = 0; i < dataTypes.length; i++) {
- if (columns[i] == null) {
- continue;
- }
- dataTypes[i].serializeTo(buffer);
- }
- }
-
- private void writeTimes(ByteBuffer buffer, int start, int end) {
- buffer.putInt(rowCount);
- for (int i = start; i < end; i++) {
- buffer.putLong(times[i]);
- }
- }
-
- private void writeBitMaps(ByteBuffer buffer, int start, int end) {
- buffer.put(BytesUtils.boolToByte(bitMaps != null));
- if (bitMaps != null) {
- for (int i = 0; i < bitMaps.length; i++) {
- if (columns[i] == null) {
- continue;
- }
-
- if (bitMaps[i] == null) {
- buffer.put(BytesUtils.boolToByte(false));
- } else {
- buffer.put(BytesUtils.boolToByte(true));
- int len = end - start;
- BitMap partBitMap = new BitMap(len);
- BitMap.copyOfRange(bitMaps[i], start, partBitMap, 0, len);
- buffer.put(partBitMap.getByteArray());
- }
- }
- }
- }
-
- private void writeValues(ByteBuffer buffer, int start, int end) {
- serializeValues(buffer, start, end);
- buffer.putLong(index);
- }
-
- private void serializeValues(ByteBuffer buffer, int start, int end) {
- for (int i = 0; i < columns.length; i++) {
- if (columns[i] == null) {
- continue;
- }
- serializeColumn(dataTypes[i], columns[i], buffer, start, end);
- }
- }
-
- private void serializeColumn(
- TSDataType dataType, Object column, ByteBuffer buffer, int start, int end) {
- switch (dataType) {
- case INT32:
- int[] intValues = (int[]) column;
- for (int j = start; j < end; j++) {
- buffer.putInt(intValues[j]);
- }
- break;
- case INT64:
- long[] longValues = (long[]) column;
- for (int j = start; j < end; j++) {
- buffer.putLong(longValues[j]);
- }
- break;
- case FLOAT:
- float[] floatValues = (float[]) column;
- for (int j = start; j < end; j++) {
- buffer.putFloat(floatValues[j]);
- }
- break;
- case DOUBLE:
- double[] doubleValues = (double[]) column;
- for (int j = start; j < end; j++) {
- buffer.putDouble(doubleValues[j]);
- }
- break;
- case BOOLEAN:
- boolean[] boolValues = (boolean[]) column;
- for (int j = start; j < end; j++) {
- buffer.put(BytesUtils.boolToByte(boolValues[j]));
- }
- break;
- case TEXT:
- Binary[] binaryValues = (Binary[]) column;
- for (int j = start; j < end; j++) {
- buffer.putInt(binaryValues[j].getLength());
- buffer.put(binaryValues[j].getValues());
- }
- break;
- default:
- throw new UnSupportedDataTypeException(String.format(DATATYPE_UNSUPPORTED, dataType));
- }
- }
-
- @Override
- public void serializeToWAL(IWALByteBufferView buffer) {
- serializeToWAL(buffer, 0, rowCount);
- }
-
- public void serializeToWAL(IWALByteBufferView buffer, int start, int end) {
- int type = PhysicalPlanType.BATCHINSERT.ordinal();
- buffer.put((byte) type);
- subSerialize(buffer, start, end);
- }
-
- void subSerialize(IWALByteBufferView buffer, int start, int end) {
- WALWriteUtils.write(devicePath.getFullPath(), buffer);
- writeMeasurements(buffer);
- writeDataTypes(buffer);
- writeTimes(buffer, start, end);
- writeBitMaps(buffer, start, end);
- writeValues(buffer, start, end);
- buffer.put((byte) (isAligned ? 1 : 0));
- }
-
- private void writeMeasurements(IWALByteBufferView buffer) {
- buffer.putInt(measurements.length - getFailedMeasurementNumber());
- for (String m : measurements) {
- if (m != null) {
- WALWriteUtils.write(m, buffer);
- }
- }
- }
-
- private void writeDataTypes(IWALByteBufferView buffer) {
- buffer.putInt(dataTypes.length - getFailedMeasurementNumber());
- for (int i = 0, dataTypesLength = dataTypes.length; i < dataTypesLength; i++) {
- TSDataType dataType = dataTypes[i];
- if (columns[i] == null) {
- continue;
- }
- WALWriteUtils.write(dataType, buffer);
- }
- }
-
- private void writeTimes(IWALByteBufferView buffer, int start, int end) {
- buffer.putInt(end - start);
- for (int i = start; i < end; i++) {
- buffer.putLong(times[i]);
- }
- }
-
- private void writeBitMaps(IWALByteBufferView buffer, int start, int end) {
- buffer.put(BytesUtils.boolToByte(bitMaps != null));
- if (bitMaps != null) {
- for (int i = 0; i < bitMaps.length; i++) {
- if (columns[i] == null) {
- continue;
- }
-
- if (bitMaps[i] == null) {
- buffer.put(BytesUtils.boolToByte(false));
- } else {
- buffer.put(BytesUtils.boolToByte(true));
- int len = end - start;
- BitMap partBitMap = new BitMap(len);
- BitMap.copyOfRange(bitMaps[i], start, partBitMap, 0, len);
- buffer.put(partBitMap.getByteArray());
- }
- }
- }
- }
-
- private void writeValues(IWALByteBufferView buffer, int start, int end) {
- serializeValues(buffer, start, end);
- buffer.putLong(index);
- }
-
- private void serializeValues(IWALByteBufferView buffer, int start, int end) {
- for (int i = 0; i < columns.length; i++) {
- if (columns[i] == null) {
- continue;
- }
- serializeColumn(dataTypes[i], columns[i], buffer, start, end);
- }
- }
-
- private void serializeColumn(
- TSDataType dataType, Object column, IWALByteBufferView buffer, int start, int end) {
- switch (dataType) {
- case INT32:
- int[] intValues = (int[]) column;
- for (int j = start; j < end; j++) {
- buffer.putInt(intValues[j]);
- }
- break;
- case INT64:
- long[] longValues = (long[]) column;
- for (int j = start; j < end; j++) {
- buffer.putLong(longValues[j]);
- }
- break;
- case FLOAT:
- float[] floatValues = (float[]) column;
- for (int j = start; j < end; j++) {
- buffer.putFloat(floatValues[j]);
- }
- break;
- case DOUBLE:
- double[] doubleValues = (double[]) column;
- for (int j = start; j < end; j++) {
- buffer.putDouble(doubleValues[j]);
- }
- break;
- case BOOLEAN:
- boolean[] boolValues = (boolean[]) column;
- for (int j = start; j < end; j++) {
- buffer.put(BytesUtils.boolToByte(boolValues[j]));
- }
- break;
- case TEXT:
- Binary[] binaryValues = (Binary[]) column;
- for (int j = start; j < end; j++) {
- buffer.putInt(binaryValues[j].getLength());
- buffer.put(binaryValues[j].getValues());
- }
- break;
- default:
- throw new UnSupportedDataTypeException(String.format(DATATYPE_UNSUPPORTED, dataType));
- }
- }
-
- @Override
- public void deserialize(DataInputStream stream) throws IOException, IllegalPathException {
- this.devicePath = new PartialPath(ReadWriteIOUtils.readString(stream));
-
- int measurementSize = stream.readInt();
- this.measurements = new String[measurementSize];
- for (int i = 0; i < measurementSize; i++) {
- measurements[i] = ReadWriteIOUtils.readString(stream);
- }
-
- int dataTypeSize = stream.readInt();
- this.dataTypes = new TSDataType[dataTypeSize];
- for (int i = 0; i < dataTypeSize; i++) {
- dataTypes[i] = TSDataType.deserialize(stream.readByte());
- }
-
- int rows = stream.readInt();
- rowCount = rows;
- this.times = new long[rows];
- times = QueryDataSetUtils.readTimesFromStream(stream, rows);
-
- boolean hasBitMaps = BytesUtils.byteToBool(stream.readByte());
- if (hasBitMaps) {
- bitMaps = QueryDataSetUtils.readBitMapsFromStream(stream, dataTypeSize, rows);
- }
- columns = QueryDataSetUtils.readTabletValuesFromStream(stream, dataTypes, dataTypeSize, rows);
- this.index = stream.readLong();
- this.isAligned = stream.readByte() == 1;
- }
-
- @Override
- public void deserialize(ByteBuffer buffer) throws IllegalPathException {
- this.devicePath = new PartialPath(readString(buffer));
-
- int measurementSize = buffer.getInt();
- this.measurements = new String[measurementSize];
- for (int i = 0; i < measurementSize; i++) {
- measurements[i] = readString(buffer);
- }
-
- int dataTypeSize = buffer.getInt();
- this.dataTypes = new TSDataType[dataTypeSize];
- for (int i = 0; i < dataTypeSize; i++) {
- dataTypes[i] = TSDataType.deserialize(buffer.get());
- }
-
- int rows = buffer.getInt();
- rowCount = rows;
- this.times = new long[rows];
- times = QueryDataSetUtils.readTimesFromBuffer(buffer, rows);
-
- boolean hasBitMaps = BytesUtils.byteToBool(buffer.get());
- if (hasBitMaps) {
- bitMaps = QueryDataSetUtils.readBitMapsFromBuffer(buffer, dataTypeSize, rows);
- }
- columns = QueryDataSetUtils.readTabletValuesFromBuffer(buffer, dataTypes, dataTypeSize, rows);
- this.index = buffer.getLong();
- this.isAligned = buffer.get() == 1;
- }
-
- public void setDataTypes(List<Integer> dataTypes) {
- this.dataTypes = new TSDataType[dataTypes.size()];
- for (int i = 0; i < dataTypes.size(); i++) {
- this.dataTypes[i] = TSDataType.values()[dataTypes.get(i)];
- }
- }
-
- public Object[] getColumns() {
- return columns;
- }
-
- public void setColumns(Object[] columns) {
- this.columns = columns;
- }
-
- public void setColumn(int index, Object column) {
- columns[index] = column;
- }
-
- public BitMap[] getBitMaps() {
- return bitMaps;
- }
-
- public void setBitMaps(BitMap[] bitMaps) {
- this.bitMaps = bitMaps;
- }
-
- @Override
- public long getMinTime() {
- return times.length != 0 ? times[0] : Long.MIN_VALUE;
- }
-
- public long getMaxTime() {
- return times.length != 0 ? times[times.length - 1] : Long.MAX_VALUE;
- }
-
- @Override
- public Object getFirstValueOfIndex(int index) {
- Object value;
- switch (dataTypes[index]) {
- case INT32:
- int[] intValues = (int[]) columns[index];
- value = intValues[0];
- break;
- case INT64:
- long[] longValues = (long[]) columns[index];
- value = longValues[0];
- break;
- case FLOAT:
- float[] floatValues = (float[]) columns[index];
- value = floatValues[0];
- break;
- case DOUBLE:
- double[] doubleValues = (double[]) columns[index];
- value = doubleValues[0];
- break;
- case BOOLEAN:
- boolean[] boolValues = (boolean[]) columns[index];
- value = boolValues[0];
- break;
- case TEXT:
- Binary[] binaryValues = (Binary[]) columns[index];
- value = binaryValues[0];
- break;
- default:
- throw new UnSupportedDataTypeException(
- String.format(DATATYPE_UNSUPPORTED, dataTypes[index]));
- }
- return value;
- }
-
- public TimeValuePair composeLastTimeValuePair(int measurementIndex) {
- if (measurementIndex >= columns.length) {
- return null;
- }
-
- // get non-null value
- int lastIdx = rowCount - 1;
- if (bitMaps != null && bitMaps[measurementIndex] != null) {
- BitMap bitMap = bitMaps[measurementIndex];
- while (lastIdx >= 0) {
- if (!bitMap.isMarked(lastIdx)) {
- break;
- }
- lastIdx--;
- }
- }
- if (lastIdx < 0) {
- return null;
- }
-
- TsPrimitiveType value;
- switch (dataTypes[measurementIndex]) {
- case INT32:
- int[] intValues = (int[]) columns[measurementIndex];
- value = new TsInt(intValues[lastIdx]);
- break;
- case INT64:
- long[] longValues = (long[]) columns[measurementIndex];
- value = new TsLong(longValues[lastIdx]);
- break;
- case FLOAT:
- float[] floatValues = (float[]) columns[measurementIndex];
- value = new TsFloat(floatValues[lastIdx]);
- break;
- case DOUBLE:
- double[] doubleValues = (double[]) columns[measurementIndex];
- value = new TsDouble(doubleValues[lastIdx]);
- break;
- case BOOLEAN:
- boolean[] boolValues = (boolean[]) columns[measurementIndex];
- value = new TsBoolean(boolValues[lastIdx]);
- break;
- case TEXT:
- Binary[] binaryValues = (Binary[]) columns[measurementIndex];
- value = new TsBinary(binaryValues[lastIdx]);
- break;
- default:
- throw new UnSupportedDataTypeException(
- String.format(DATATYPE_UNSUPPORTED, dataTypes[measurementIndex]));
- }
- return new TimeValuePair(times[lastIdx], value);
- }
-
- public long[] getTimes() {
- return times;
- }
-
- public void setTimes(long[] times) {
- this.times = times;
- }
-
- public int getRowCount() {
- return rowCount;
- }
-
- public void setRowCount(int size) {
- this.rowCount = size;
- }
-
- @Override
- public String toString() {
- return "InsertTabletPlan {"
- + "prefixPath:"
- + devicePath
- + ", timesRange["
- + times[0]
- + ","
- + times[times.length - 1]
- + "]"
- + ", isAligned:"
- + isAligned
- + '}';
- }
-
- @Override
- public void markFailedMeasurementInsertion(int index, Exception e) {
- if (measurements[index] == null) {
- return;
- }
- super.markFailedMeasurementInsertion(index, e);
- if (failedColumns == null) {
- failedColumns = new ArrayList<>();
- }
- failedColumns.add(columns[index]);
- columns[index] = null;
- }
-
- @Override
- public InsertPlan getPlanFromFailed() {
- if (super.getPlanFromFailed() == null) {
- return null;
- }
- // TODO anything else?
- columns = failedColumns.toArray(new Object[0]);
- failedColumns = null;
- return this;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- InsertTabletPlan that = (InsertTabletPlan) o;
-
- return rowCount == that.rowCount
- && Objects.equals(devicePath, that.devicePath)
- && Arrays.equals(times, that.times)
- && Objects.equals(paths, that.paths)
- && Objects.equals(range, that.range)
- && Objects.equals(isAligned, that.isAligned);
- }
-
- @Override
- public int hashCode() {
- int result = Objects.hash(rowCount, paths, range);
- result = 31 * result + Arrays.hashCode(times);
- return result;
- }
-
- @Override
- public void recoverFromFailure() {
- if (failedMeasurements == null) {
- return;
- }
-
- for (int i = 0; i < failedMeasurements.size(); i++) {
- int index = failedIndices.get(i);
- columns[index] = failedColumns.get(i);
- }
- super.recoverFromFailure();
-
- failedColumns = null;
- }
-
- @Override
- public void checkIntegrity() throws QueryProcessException {
- super.checkIntegrity();
- if (columns == null || columns.length == 0) {
- throw new QueryProcessException("Values are null");
- }
- if (dataTypes.length != columns.length) {
- throw new QueryProcessException(
- String.format(
- "Measurements length [%d] does not match " + "columns length [%d]",
- measurements.length, columns.length));
- }
- for (Object value : columns) {
- if (value == null) {
- throw new QueryProcessException("Columns contain null: " + Arrays.toString(columns));
- }
- }
- }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/ClientRPCServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/ClientRPCServiceImpl.java
index e002d89ad5..b712c13779 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/ClientRPCServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/ClientRPCServiceImpl.java
@@ -953,7 +953,7 @@ public class ClientRPCServiceImpl implements IClientRPCServiceWithHandler {
// check whether measurement is legal according to syntax convention
PathUtils.isLegalSingleMeasurementLists(req.getMeasurementsList());
- // Step 1: TODO(INSERT) transfer from TSInsertTabletsReq to Statement
+ // Step 1: transfer from TSInsertRecordsReq to Statement
InsertRowsStatement statement = (InsertRowsStatement) StatementGenerator.createStatement(req);
// return success when this statement is empty because server doesn't need to execute it
if (statement.isEmpty()) {
@@ -1008,7 +1008,7 @@ public class ClientRPCServiceImpl implements IClientRPCServiceWithHandler {
// check whether measurement is legal according to syntax convention
PathUtils.isLegalSingleMeasurementLists(req.getMeasurementsList());
- // Step 1: TODO(INSERT) transfer from TSInsertTabletsReq to Statement
+ // Step 1: transfer from TSInsertRecordsOfOneDeviceReq to Statement
InsertRowsOfOneDeviceStatement statement =
(InsertRowsOfOneDeviceStatement) StatementGenerator.createStatement(req);
// return success when this statement is empty because server doesn't need to execute it
@@ -1064,7 +1064,7 @@ public class ClientRPCServiceImpl implements IClientRPCServiceWithHandler {
// check whether measurement is legal according to syntax convention
PathUtils.isLegalSingleMeasurementLists(req.getMeasurementsList());
- // Step 1: TODO(INSERT) transfer from TSInsertTabletsReq to Statement
+ // Step 1: transfer from TSInsertStringRecordsOfOneDeviceReq to Statement
InsertRowsOfOneDeviceStatement statement =
(InsertRowsOfOneDeviceStatement) StatementGenerator.createStatement(req);
// return success when this statement is empty because server doesn't need to execute it
@@ -1166,7 +1166,7 @@ public class ClientRPCServiceImpl implements IClientRPCServiceWithHandler {
PathUtils.isLegalSingleMeasurementLists(req.getMeasurementsList());
- // Step 1: TODO(INSERT) transfer from TSInsertTabletsReq to Statement
+ // Step 1: transfer from TSInsertTabletsReq to Statement
InsertMultiTabletsStatement statement =
(InsertMultiTabletsStatement) StatementGenerator.createStatement(req);
// return success when this statement is empty because server doesn't need to execute it
@@ -1213,7 +1213,7 @@ public class ClientRPCServiceImpl implements IClientRPCServiceWithHandler {
// check whether measurement is legal according to syntax convention
PathUtils.isLegalSingleMeasurements(req.getMeasurements());
- // Step 1: TODO(INSERT) transfer from TSInsertTabletReq to Statement
+ // Step 1: transfer from TSInsertTabletReq to Statement
InsertTabletStatement statement =
(InsertTabletStatement) StatementGenerator.createStatement(req);
// return success when this statement is empty because server doesn't need to execute it
diff --git a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/TSServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/TSServiceImpl.java
index 72f1112c71..bddca8c86c 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/TSServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/TSServiceImpl.java
@@ -37,18 +37,11 @@ import org.apache.iotdb.db.auth.AuthorizerManager;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.conf.OperationType;
-import org.apache.iotdb.db.exception.QueryInBatchStatementException;
import org.apache.iotdb.db.exception.StorageEngineException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.metadata.template.TemplateQueryType;
import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertMultiTabletsPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowsOfOneDevicePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowsPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
import org.apache.iotdb.db.qp.physical.crud.UDFPlan;
import org.apache.iotdb.db.qp.physical.sys.AppendTemplatePlan;
@@ -366,25 +359,6 @@ public class TSServiceImpl implements IClientRPCServiceWithHandler {
return IoTDB.schemaProcessor.getSeriesType(path);
}
- private boolean executeInsertRowsPlan(InsertRowsPlan insertRowsPlan, List<TSStatus> result) {
- long t1 = System.currentTimeMillis();
- TSStatus tsStatus = executeNonQueryPlan(insertRowsPlan);
- addOperationLatency(Operation.EXECUTE_ROWS_PLAN_IN_BATCH, t1);
- int startIndex = result.size();
- if (startIndex > 0) {
- startIndex = startIndex - 1;
- }
- for (int i = 0; i < insertRowsPlan.getRowCount(); i++) {
- result.add(RpcUtils.SUCCESS_STATUS);
- }
- if (tsStatus.subStatus != null) {
- for (Entry<Integer, TSStatus> entry : insertRowsPlan.getResults().entrySet()) {
- result.set(startIndex + entry.getKey(), entry.getValue());
- }
- }
- return tsStatus.getCode() == RpcUtils.SUCCESS_STATUS.getCode();
- }
-
private boolean executeMultiTimeSeriesPlan(
CreateMultiTimeSeriesPlan multiPlan, List<TSStatus> result) {
long t1 = System.currentTimeMillis();
@@ -444,130 +418,9 @@ public class TSServiceImpl implements IClientRPCServiceWithHandler {
multiPlan.getAlias().add(alias);
}
- private boolean executeBatchList(List executeList, List<TSStatus> result) {
- boolean isAllSuccessful = true;
- for (int j = 0; j < executeList.size(); j++) {
- Object planObject = executeList.get(j);
- if (InsertRowsPlan.class.isInstance(planObject)) {
- if (!executeInsertRowsPlan((InsertRowsPlan) planObject, result)) {
- isAllSuccessful = false;
- }
- } else if (CreateMultiTimeSeriesPlan.class.isInstance(planObject)) {
- if (!executeMultiTimeSeriesPlan((CreateMultiTimeSeriesPlan) planObject, result)) {
- isAllSuccessful = false;
- }
- }
- }
- return isAllSuccessful;
- }
-
@Override
public TSStatus executeBatchStatement(TSExecuteBatchStatementReq req) {
- long t1 = System.currentTimeMillis();
- List<TSStatus> result = new ArrayList<>();
- boolean isAllSuccessful = true;
- if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
- return getNotLoggedInStatus();
- }
-
- InsertRowsPlan insertRowsPlan;
- int index = 0;
- List<Object> executeList = new ArrayList<>();
- OperatorType lastOperatorType = null;
- CreateMultiTimeSeriesPlan multiPlan;
- for (int i = 0; i < req.getStatements().size(); i++) {
- String statement = req.getStatements().get(i);
- try {
- PhysicalPlan physicalPlan =
- serviceProvider
- .getPlanner()
- .parseSQLToPhysicalPlan(
- statement,
- SESSION_MANAGER.getSessionTimeZone().toZoneId(),
- SESSION_MANAGER.getCurrSession().getClientVersion());
- if (physicalPlan.isQuery() || physicalPlan.isSelectInto()) {
- throw new QueryInBatchStatementException(statement);
- }
-
- if (physicalPlan.getOperatorType().equals(OperatorType.INSERT)) {
- if (OperatorType.INSERT == lastOperatorType) {
- insertRowsPlan = (InsertRowsPlan) executeList.get(executeList.size() - 1);
- } else {
- insertRowsPlan = new InsertRowsPlan();
- executeList.add(insertRowsPlan);
- index = 0;
- }
-
- TSStatus status =
- SESSION_MANAGER.checkAuthority(physicalPlan, SESSION_MANAGER.getCurrSession());
- if (status != null) {
- insertRowsPlan.getResults().put(index, status);
- isAllSuccessful = false;
- }
-
- lastOperatorType = OperatorType.INSERT;
- insertRowsPlan.addOneInsertRowPlan((InsertRowPlan) physicalPlan, index);
- index++;
-
- if (i == req.getStatements().size() - 1) {
- if (!executeBatchList(executeList, result)) {
- isAllSuccessful = false;
- }
- }
- } else if (physicalPlan.getOperatorType().equals(OperatorType.CREATE_TIMESERIES)) {
- if (OperatorType.CREATE_TIMESERIES == lastOperatorType) {
- multiPlan = (CreateMultiTimeSeriesPlan) executeList.get(executeList.size() - 1);
- } else {
- multiPlan = new CreateMultiTimeSeriesPlan();
- executeList.add(multiPlan);
- }
- TSStatus status =
- SESSION_MANAGER.checkAuthority(physicalPlan, SESSION_MANAGER.getCurrSession());
- if (status != null) {
- multiPlan.getResults().put(i, status);
- isAllSuccessful = false;
- }
-
- lastOperatorType = OperatorType.CREATE_TIMESERIES;
- initMultiTimeSeriesPlan(multiPlan);
-
- CreateTimeSeriesPlan createTimeSeriesPlan = (CreateTimeSeriesPlan) physicalPlan;
- setMultiTimeSeriesPlan(multiPlan, createTimeSeriesPlan);
- if (i == req.getStatements().size() - 1) {
- if (!executeBatchList(executeList, result)) {
- isAllSuccessful = false;
- }
- }
- } else {
- lastOperatorType = physicalPlan.getOperatorType();
- if (!executeList.isEmpty()) {
- if (!executeBatchList(executeList, result)) {
- isAllSuccessful = false;
- }
- executeList.clear();
- }
- long t2 = System.currentTimeMillis();
- TSExecuteStatementResp resp = executeNonQueryStatement(physicalPlan);
- addOperationLatency(Operation.EXECUTE_ONE_SQL_IN_BATCH, t2);
- result.add(resp.status);
- if (resp.getStatus().code != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
- isAllSuccessful = false;
- }
- }
- } catch (Exception e) {
- LOGGER.error("Error occurred when executing executeBatchStatement: ", e);
- TSStatus status =
- onQueryException(e, "\"" + statement + "\". " + OperationType.EXECUTE_BATCH_STATEMENT);
- if (status.getCode() != TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()) {
- isAllSuccessful = false;
- }
- result.add(status);
- }
- }
- addOperationLatency(Operation.EXECUTE_JDBC_BATCH, t1);
- return isAllSuccessful
- ? RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS, "Execute batch statements successfully")
- : RpcUtils.getStatus(result);
+ return null;
}
@Override
@@ -886,23 +739,6 @@ public class TSServiceImpl implements IClientRPCServiceWithHandler {
return resp;
}
- private TSStatus insertTabletsInternally(List<InsertTabletPlan> insertTabletPlans) {
- InsertMultiTabletsPlan insertMultiTabletsPlan = new InsertMultiTabletsPlan();
- for (int i = 0; i < insertTabletPlans.size(); i++) {
- InsertTabletPlan insertTabletPlan = insertTabletPlans.get(i);
- TSStatus status =
- SESSION_MANAGER.checkAuthority(insertTabletPlan, SESSION_MANAGER.getCurrSession());
-
- if (status != null) {
- // not authorized
- insertMultiTabletsPlan.getResults().put(i, status);
- }
- }
- insertMultiTabletsPlan.setInsertTabletPlanList(insertTabletPlans);
-
- return executeNonQueryPlan(insertMultiTabletsPlan);
- }
-
@SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
@Override
public TSFetchResultsResp fetchResults(TSFetchResultsReq req) {
@@ -1085,56 +921,7 @@ public class TSServiceImpl implements IClientRPCServiceWithHandler {
@Override
public TSStatus insertRecords(TSInsertRecordsReq req) {
- if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
- return getNotLoggedInStatus();
- }
-
- if (AUDIT_LOGGER.isDebugEnabled()) {
- AUDIT_LOGGER.debug(
- "Session {} insertRecords, first device {}, first time {}",
- SESSION_MANAGER.getCurrSession(),
- req.prefixPaths.get(0),
- req.getTimestamps().get(0));
- }
- boolean allCheckSuccess = true;
-
- InsertRowsPlan insertRowsPlan = new InsertRowsPlan();
- for (int i = 0; i < req.prefixPaths.size(); i++) {
- try {
- // check whether measurement is legal according to syntax convention
- PathUtils.isLegalSingleMeasurements(req.getMeasurementsList().get(i));
- InsertRowPlan plan =
- new InsertRowPlan(
- new PartialPath(req.getPrefixPaths().get(i)),
- req.getTimestamps().get(i),
- req.getMeasurementsList().get(i).toArray(new String[0]),
- req.valuesList.get(i),
- req.isAligned);
- TSStatus status = SESSION_MANAGER.checkAuthority(plan, SESSION_MANAGER.getCurrSession());
- if (status != null) {
- insertRowsPlan.getResults().put(i, status);
- allCheckSuccess = false;
- }
- insertRowsPlan.addOneInsertRowPlan(plan, i);
- } catch (IoTDBException e) {
- allCheckSuccess = false;
- insertRowsPlan
- .getResults()
- .put(i, onIoTDBException(e, OperationType.INSERT_RECORDS, e.getErrorCode()));
- } catch (Exception e) {
- allCheckSuccess = false;
- insertRowsPlan
- .getResults()
- .put(
- i,
- onNPEOrUnexpectedException(
- e, OperationType.INSERT_RECORDS, TSStatusCode.INTERNAL_SERVER_ERROR));
- }
- }
- TSStatus tsStatus = executeNonQueryPlan(insertRowsPlan);
-
- return judgeFinalTsStatus(
- allCheckSuccess, tsStatus, insertRowsPlan.getResults(), req.prefixPaths.size());
+ return null;
}
private TSStatus judgeFinalTsStatus(
@@ -1160,183 +947,17 @@ public class TSServiceImpl implements IClientRPCServiceWithHandler {
@Override
public TSStatus insertRecordsOfOneDevice(TSInsertRecordsOfOneDeviceReq req) {
- if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
- return getNotLoggedInStatus();
- }
-
- if (AUDIT_LOGGER.isDebugEnabled()) {
- AUDIT_LOGGER.debug(
- "Session {} insertRecords, device {}, first time {}",
- SESSION_MANAGER.getCurrSession(),
- req.prefixPath,
- req.getTimestamps().get(0));
- }
-
- List<TSStatus> statusList = new ArrayList<>();
- try {
- // check whether measurement is legal according to syntax convention
- PathUtils.isLegalSingleMeasurementLists(req.getMeasurementsList());
- InsertRowsOfOneDevicePlan plan =
- new InsertRowsOfOneDevicePlan(
- new PartialPath(req.getPrefixPath()),
- req.getTimestamps(),
- req.getMeasurementsList(),
- req.getValuesList(),
- req.isAligned);
- TSStatus status = SESSION_MANAGER.checkAuthority(plan, SESSION_MANAGER.getCurrSession());
- statusList.add(status != null ? status : executeNonQueryPlan(plan));
- } catch (IoTDBException e) {
- statusList.add(
- onIoTDBException(e, OperationType.INSERT_RECORDS_OF_ONE_DEVICE, e.getErrorCode()));
- } catch (Exception e) {
- statusList.add(
- onNPEOrUnexpectedException(
- e, OperationType.INSERT_RECORDS_OF_ONE_DEVICE, TSStatusCode.INTERNAL_SERVER_ERROR));
- }
-
- TSStatus resp = RpcUtils.getStatus(statusList);
- for (TSStatus status : resp.subStatus) {
- if (status.code != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
- return resp;
- }
- }
-
- resp.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode());
-
- return resp;
+ return null;
}
@Override
public TSStatus insertStringRecordsOfOneDevice(TSInsertStringRecordsOfOneDeviceReq req) {
- if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
- return getNotLoggedInStatus();
- }
-
- if (AUDIT_LOGGER.isDebugEnabled()) {
- AUDIT_LOGGER.debug(
- "Session {} insertRecords, device {}, first time {}",
- SESSION_MANAGER.getCurrSession(),
- req.prefixPath,
- req.getTimestamps().get(0));
- }
-
- boolean allCheckSuccess = true;
- InsertRowsPlan insertRowsPlan = new InsertRowsPlan();
- for (int i = 0; i < req.timestamps.size(); i++) {
- InsertRowPlan plan = new InsertRowPlan();
- try {
- // check whether measurement is legal according to syntax convention
- PathUtils.isLegalSingleMeasurements(req.getMeasurementsList().get(i));
- plan.setDevicePath(new PartialPath(req.getPrefixPath()));
- plan.setTime(req.getTimestamps().get(i));
- addMeasurementAndValue(plan, req.getMeasurementsList().get(i), req.getValuesList().get(i));
- plan.setDataTypes(new TSDataType[plan.getMeasurements().length]);
- plan.setNeedInferType(true);
- plan.setAligned(req.isAligned);
- TSStatus status = SESSION_MANAGER.checkAuthority(plan, SESSION_MANAGER.getCurrSession());
-
- if (status != null) {
- insertRowsPlan.getResults().put(i, status);
- allCheckSuccess = false;
- }
- insertRowsPlan.addOneInsertRowPlan(plan, i);
- } catch (IoTDBException e) {
- insertRowsPlan
- .getResults()
- .put(
- i,
- onIoTDBException(
- e, OperationType.INSERT_STRING_RECORDS_OF_ONE_DEVICE, e.getErrorCode()));
- allCheckSuccess = false;
- } catch (Exception e) {
- insertRowsPlan
- .getResults()
- .put(
- i,
- onNPEOrUnexpectedException(
- e,
- OperationType.INSERT_STRING_RECORDS_OF_ONE_DEVICE,
- TSStatusCode.INTERNAL_SERVER_ERROR));
- allCheckSuccess = false;
- }
- }
- TSStatus tsStatus = executeNonQueryPlan(insertRowsPlan);
-
- return judgeFinalTsStatus(
- allCheckSuccess, tsStatus, insertRowsPlan.getResults(), req.timestamps.size());
+ return null;
}
@Override
public TSStatus insertStringRecords(TSInsertStringRecordsReq req) {
- if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
- return getNotLoggedInStatus();
- }
-
- if (AUDIT_LOGGER.isDebugEnabled()) {
- AUDIT_LOGGER.debug(
- "Session {} insertRecords, first device {}, first time {}",
- SESSION_MANAGER.getCurrSession(),
- req.prefixPaths.get(0),
- req.getTimestamps().get(0));
- }
-
- boolean allCheckSuccess = true;
- InsertRowsPlan insertRowsPlan = new InsertRowsPlan();
- for (int i = 0; i < req.prefixPaths.size(); i++) {
- InsertRowPlan plan = new InsertRowPlan();
- try {
- // check whether measurement is legal according to syntax convention
- PathUtils.isLegalSingleMeasurements(req.getMeasurementsList().get(i));
- plan.setDevicePath(new PartialPath(req.getPrefixPaths().get(i)));
- plan.setTime(req.getTimestamps().get(i));
- addMeasurementAndValue(plan, req.getMeasurementsList().get(i), req.getValuesList().get(i));
- plan.setDataTypes(new TSDataType[plan.getMeasurements().length]);
- plan.setNeedInferType(true);
- plan.setAligned(req.isAligned);
- TSStatus status = SESSION_MANAGER.checkAuthority(plan, SESSION_MANAGER.getCurrSession());
-
- if (status != null) {
- insertRowsPlan.getResults().put(i, status);
- allCheckSuccess = false;
- }
- insertRowsPlan.addOneInsertRowPlan(plan, i);
- } catch (IoTDBException e) {
- insertRowsPlan
- .getResults()
- .put(i, onIoTDBException(e, OperationType.INSERT_STRING_RECORDS, e.getErrorCode()));
- allCheckSuccess = false;
- } catch (Exception e) {
- insertRowsPlan
- .getResults()
- .put(
- i,
- onNPEOrUnexpectedException(
- e, OperationType.INSERT_STRING_RECORDS, TSStatusCode.INTERNAL_SERVER_ERROR));
- allCheckSuccess = false;
- }
- }
- TSStatus tsStatus = executeNonQueryPlan(insertRowsPlan);
-
- return judgeFinalTsStatus(
- allCheckSuccess, tsStatus, insertRowsPlan.getResults(), req.prefixPaths.size());
- }
-
- private void addMeasurementAndValue(
- InsertRowPlan insertRowPlan, List<String> measurements, List<String> values) {
- List<String> newMeasurements = new ArrayList<>(measurements.size());
- List<Object> newValues = new ArrayList<>(values.size());
-
- for (int i = 0; i < measurements.size(); ++i) {
- String value = values.get(i);
- if (value.isEmpty()) {
- continue;
- }
- newMeasurements.add(measurements.get(i));
- newValues.add(value);
- }
-
- insertRowPlan.setValues(newValues.toArray(new Object[0]));
- insertRowPlan.setMeasurements(newMeasurements.toArray(new String[0]));
+ return null;
}
@Override
@@ -1383,133 +1004,22 @@ public class TSServiceImpl implements IClientRPCServiceWithHandler {
@Override
public TSStatus insertRecord(TSInsertRecordReq req) {
- try {
- if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
- return getNotLoggedInStatus();
- }
-
- AUDIT_LOGGER.debug(
- "Session {} insertRecord, device {}, time {}",
- SESSION_MANAGER.getCurrSession(),
- req.getPrefixPath(),
- req.getTimestamp());
-
- // check whether measurement is legal according to syntax convention
-
- PathUtils.isLegalSingleMeasurements(req.getMeasurements());
-
- InsertRowPlan plan =
- new InsertRowPlan(
- new PartialPath(req.getPrefixPath()),
- req.getTimestamp(),
- req.getMeasurements().toArray(new String[0]),
- req.values,
- req.isAligned);
- TSStatus status = SESSION_MANAGER.checkAuthority(plan, SESSION_MANAGER.getCurrSession());
- return status != null ? status : executeNonQueryPlan(plan);
- } catch (IoTDBException e) {
- return onIoTDBException(e, OperationType.INSERT_RECORD, e.getErrorCode());
- } catch (Exception e) {
- return onNPEOrUnexpectedException(
- e, OperationType.INSERT_RECORD, TSStatusCode.EXECUTE_STATEMENT_ERROR);
- }
+ return null;
}
@Override
public TSStatus insertStringRecord(TSInsertStringRecordReq req) {
- try {
- if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
- return getNotLoggedInStatus();
- }
-
- AUDIT_LOGGER.debug(
- "Session {} insertRecord, device {}, time {}",
- SESSION_MANAGER.getCurrSession(),
- req.getPrefixPath(),
- req.getTimestamp());
-
- // check whether measurement is legal according to syntax convention
- PathUtils.isLegalSingleMeasurements(req.getMeasurements());
-
- InsertRowPlan plan = new InsertRowPlan();
- plan.setDevicePath(new PartialPath(req.getPrefixPath()));
- plan.setTime(req.getTimestamp());
- plan.setMeasurements(req.getMeasurements().toArray(new String[0]));
- plan.setDataTypes(new TSDataType[plan.getMeasurements().length]);
- plan.setValues(req.getValues().toArray(new Object[0]));
- plan.setNeedInferType(true);
- plan.setAligned(req.isAligned);
- TSStatus status = SESSION_MANAGER.checkAuthority(plan, SESSION_MANAGER.getCurrSession());
- return status != null ? status : executeNonQueryPlan(plan);
- } catch (IoTDBException e) {
- return onIoTDBException(e, OperationType.INSERT_STRING_RECORD, e.getErrorCode());
- } catch (Exception e) {
- return onNPEOrUnexpectedException(
- e, OperationType.INSERT_STRING_RECORD, TSStatusCode.EXECUTE_STATEMENT_ERROR);
- }
+ return null;
}
@Override
public TSStatus deleteData(TSDeleteDataReq req) {
- try {
- if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
- return getNotLoggedInStatus();
- }
-
- DeletePlan plan = new DeletePlan();
- plan.setDeleteStartTime(req.getStartTime());
- plan.setDeleteEndTime(req.getEndTime());
- List<PartialPath> paths = new ArrayList<>();
- for (String path : req.getPaths()) {
- paths.add(new PartialPath(path));
- }
- plan.addPaths(paths);
- TSStatus status = SESSION_MANAGER.checkAuthority(plan, SESSION_MANAGER.getCurrSession());
-
- return status != null ? new TSStatus(status) : new TSStatus(executeNonQueryPlan(plan));
- } catch (IoTDBException e) {
- return onIoTDBException(e, OperationType.DELETE_DATA, e.getErrorCode());
- } catch (Exception e) {
- return onNPEOrUnexpectedException(
- e, OperationType.DELETE_DATA, TSStatusCode.EXECUTE_STATEMENT_ERROR);
- }
+ return null;
}
@Override
public TSStatus insertTablet(TSInsertTabletReq req) {
- long t1 = System.currentTimeMillis();
- try {
- if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
- return getNotLoggedInStatus();
- }
-
- // check whether measurement is legal according to syntax convention
-
- PathUtils.isLegalSingleMeasurements(req.getMeasurements());
-
- InsertTabletPlan insertTabletPlan =
- new InsertTabletPlan(new PartialPath(req.getPrefixPath()), req.measurements);
- insertTabletPlan.setTimes(QueryDataSetUtils.readTimesFromBuffer(req.timestamps, req.size));
- insertTabletPlan.setColumns(
- QueryDataSetUtils.readTabletValuesFromBuffer(
- req.values, req.types, req.types.size(), req.size));
- insertTabletPlan.setBitMaps(
- QueryDataSetUtils.readBitMapsFromBuffer(req.values, req.types.size(), req.size));
- insertTabletPlan.setRowCount(req.size);
- insertTabletPlan.setDataTypes(req.types);
- insertTabletPlan.setAligned(req.isAligned);
- TSStatus status =
- SESSION_MANAGER.checkAuthority(insertTabletPlan, SESSION_MANAGER.getCurrSession());
-
- return status != null ? status : executeNonQueryPlan(insertTabletPlan);
- } catch (IoTDBException e) {
- return onIoTDBException(e, OperationType.INSERT_TABLET, e.getErrorCode());
- } catch (Exception e) {
- return onNPEOrUnexpectedException(
- e, OperationType.INSERT_TABLET, TSStatusCode.EXECUTE_STATEMENT_ERROR);
- } finally {
- addOperationLatency(Operation.EXECUTE_RPC_BATCH_INSERT, t1);
- }
+ return null;
}
@Override
@@ -1534,46 +1044,8 @@ public class TSServiceImpl implements IClientRPCServiceWithHandler {
}
}
- private InsertTabletPlan constructInsertTabletPlan(TSInsertTabletsReq req, int i)
- throws MetadataException {
- // check whether measurement is legal according to syntax convention
- PathUtils.isLegalSingleMeasurementLists(req.getMeasurementsList());
- InsertTabletPlan insertTabletPlan =
- new InsertTabletPlan(new PartialPath(req.prefixPaths.get(i)), req.measurementsList.get(i));
- insertTabletPlan.setTimes(
- QueryDataSetUtils.readTimesFromBuffer(req.timestampsList.get(i), req.sizeList.get(i)));
- insertTabletPlan.setColumns(
- QueryDataSetUtils.readTabletValuesFromBuffer(
- req.valuesList.get(i),
- req.typesList.get(i),
- req.measurementsList.get(i).size(),
- req.sizeList.get(i)));
- insertTabletPlan.setBitMaps(
- QueryDataSetUtils.readBitMapsFromBuffer(
- req.valuesList.get(i), req.measurementsList.get(i).size(), req.sizeList.get(i)));
- insertTabletPlan.setRowCount(req.sizeList.get(i));
- insertTabletPlan.setDataTypes(req.typesList.get(i));
- insertTabletPlan.setAligned(req.isAligned);
- return insertTabletPlan;
- }
-
- /** construct one InsertMultiTabletsPlan and process it */
public TSStatus insertTabletsInternally(TSInsertTabletsReq req) throws MetadataException {
- List<InsertTabletPlan> insertTabletPlanList = new ArrayList<>();
- InsertMultiTabletsPlan insertMultiTabletsPlan = new InsertMultiTabletsPlan();
- for (int i = 0; i < req.prefixPaths.size(); i++) {
- InsertTabletPlan insertTabletPlan = constructInsertTabletPlan(req, i);
- TSStatus status =
- SESSION_MANAGER.checkAuthority(insertTabletPlan, SESSION_MANAGER.getCurrSession());
- if (status != null) {
- // not authorized
- insertMultiTabletsPlan.getResults().put(i, status);
- }
- insertTabletPlanList.add(insertTabletPlan);
- }
-
- insertMultiTabletsPlan.setInsertTabletPlanList(insertTabletPlanList);
- return executeNonQueryPlan(insertMultiTabletsPlan);
+ return null;
}
@Override
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/MemUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/MemUtils.java
index 61be14f55e..e879e3c78a 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/MemUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/MemUtils.java
@@ -20,7 +20,6 @@ package org.apache.iotdb.db.utils;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertTabletNode;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.utils.Binary;
import org.apache.iotdb.tsfile.utils.RamUsageEstimator;
@@ -135,59 +134,6 @@ public class MemUtils {
return memSize;
}
- /**
- * If mem control enabled, do not add text data size here, the size will be added to memtable
- * before inserting.
- */
- public static long getTabletSize(
- InsertTabletPlan insertTabletPlan, int start, int end, boolean addingTextDataSize) {
- if (start >= end) {
- return 0L;
- }
- long memSize = 0;
- for (int i = 0; i < insertTabletPlan.getMeasurements().length; i++) {
- if (insertTabletPlan.getMeasurements()[i] == null) {
- continue;
- }
- // time column memSize
- memSize += (end - start) * 8L;
- if (insertTabletPlan.getDataTypes()[i] == TSDataType.TEXT && addingTextDataSize) {
- for (int j = start; j < end; j++) {
- memSize += getBinarySize(((Binary[]) insertTabletPlan.getColumns()[i])[j]);
- }
- } else {
- memSize += (end - start) * insertTabletPlan.getDataTypes()[i].getDataTypeSize();
- }
- }
- return memSize;
- }
-
- public static long getAlignedTabletSize(
- InsertTabletPlan insertTabletPlan, int start, int end, boolean addingTextDataSize) {
- if (start >= end) {
- return 0L;
- }
- long memSize = 0;
- for (int i = 0; i < insertTabletPlan.getMeasurements().length; i++) {
- if (insertTabletPlan.getMeasurements()[i] == null) {
- continue;
- }
- TSDataType valueType;
- // value columns memSize
- valueType = insertTabletPlan.getDataTypes()[i];
- if (valueType == TSDataType.TEXT && addingTextDataSize) {
- for (int j = start; j < end; j++) {
- memSize += getBinarySize(((Binary[]) insertTabletPlan.getColumns()[i])[j]);
- }
- } else {
- memSize += (long) (end - start) * valueType.getDataTypeSize();
- }
- }
- // time and index column memSize for vector
- memSize += (end - start) * (8L + 4L);
- return memSize;
- }
-
public static long getAlignedTabletSize(
InsertTabletNode insertTabletNode, int start, int end, boolean addingTextDataSize) {
if (start >= end) {
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntryType.java b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntryType.java
index ff1da0c37a..97214848de 100644
--- a/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntryType.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntryType.java
@@ -21,16 +21,12 @@ package org.apache.iotdb.db.wal.buffer;
/** Type of {@link WALEntry}, including info type and signal type */
public enum WALEntryType {
// region info entry type
- /** {@link org.apache.iotdb.db.qp.physical.crud.InsertRowPlan} */
@Deprecated
INSERT_ROW_PLAN((byte) 0),
- /** {@link org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan} */
@Deprecated
INSERT_TABLET_PLAN((byte) 1),
- /** {@link org.apache.iotdb.db.qp.physical.crud.DeletePlan} */
@Deprecated
DELETE_PLAN((byte) 2),
- /** snapshot of {@link org.apache.iotdb.db.engine.memtable.IMemTable} */
MEMORY_TABLE_SNAPSHOT((byte) 3),
/** {@link org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowNode} */
INSERT_ROW_NODE((byte) 4),
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALInfoEntry.java b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALInfoEntry.java
index d4246616c7..6b8ebb4a70 100644
--- a/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALInfoEntry.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALInfoEntry.java
@@ -30,7 +30,7 @@ public class WALInfoEntry extends WALEntry {
/** wal entry type 1 byte, memTable id 8 bytes */
public static final int FIXED_SERIALIZED_SIZE = Byte.BYTES + Long.BYTES;
- /** extra info for InsertTabletPlan type value */
+ /** extra info for InsertTablet type value */
private TabletInfo tabletInfo;
public WALInfoEntry(long memTableId, WALEntryValue value, boolean wait) {
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoer.java b/server/src/main/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoer.java
index 9f992a9442..b38b0cc6f4 100644
--- a/server/src/main/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoer.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoer.java
@@ -18,7 +18,6 @@
*/
package org.apache.iotdb.db.wal.recover.file;
-import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.memtable.IMemTable;
@@ -26,17 +25,13 @@ import org.apache.iotdb.db.engine.memtable.PrimitiveMemTable;
import org.apache.iotdb.db.engine.modification.Deletion;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.exception.WriteProcessException;
-import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
-import org.apache.iotdb.db.exception.metadata.PathNotExistException;
import org.apache.iotdb.db.metadata.idtable.IDTable;
import org.apache.iotdb.db.metadata.idtable.entry.DeviceIDFactory;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
import org.apache.iotdb.db.mpp.plan.analyze.SchemaValidator;
import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.DeleteDataNode;
import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertNode;
import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowNode;
import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertTabletNode;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -137,29 +132,6 @@ public class TsFilePlanRedoer {
}
}
- private void checkDataTypeAndMarkFailed(final IMeasurementMNode[] mNodes, InsertPlan tPlan) {
- for (int i = 0; i < mNodes.length; i++) {
- if (mNodes[i] == null) {
- tPlan.markFailedMeasurementInsertion(
- i,
- new PathNotExistException(
- tPlan.getDevicePath().getFullPath()
- + IoTDBConstant.PATH_SEPARATOR
- + tPlan.getMeasurements()[i]));
- } else if (mNodes[i].getSchema().getType() != tPlan.getDataTypes()[i]) {
- tPlan.markFailedMeasurementInsertion(
- i,
- new DataTypeMismatchException(
- tPlan.getDevicePath().getFullPath(),
- mNodes[i].getName(),
- tPlan.getDataTypes()[i],
- mNodes[i].getSchema().getType(),
- tPlan.getMinTime(),
- tPlan.getFirstValueOfIndex(i)));
- }
- }
- }
-
void resetRecoveryMemTable(IMemTable memTable) {
this.recoveryMemTable = memTable;
}
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/SchemaBasicTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/SchemaBasicTest.java
index 5de4a0ac69..b1d5e2469a 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/SchemaBasicTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/SchemaBasicTest.java
@@ -30,8 +30,6 @@ import org.apache.iotdb.db.metadata.mnode.IMNode;
import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
import org.apache.iotdb.db.metadata.template.Template;
import org.apache.iotdb.db.metadata.utils.MetaUtils;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
import org.apache.iotdb.db.qp.physical.sys.AppendTemplatePlan;
import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
@@ -1957,161 +1955,6 @@ public abstract class SchemaBasicTest {
assertTrue(schemaProcessor.isPathExist(new PartialPath("root.group_with_hyphen")));
}
- @Test
- public void testCreateAlignedTimeseriesAndInsertWithMismatchDataType() {
- LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
- try {
- schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
- schemaProcessor.createAlignedTimeSeries(
- new PartialPath("root.laptop.d1.aligned_device"),
- Arrays.asList("s1", "s2", "s3"),
- Arrays.asList(
- TSDataType.valueOf("FLOAT"),
- TSDataType.valueOf("INT64"),
- TSDataType.valueOf("INT32")),
- Arrays.asList(
- TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE")),
- Arrays.asList(compressionType, compressionType, compressionType));
-
- // construct an insertRowPlan with mismatched data type
- long time = 1L;
- TSDataType[] dataTypes =
- new TSDataType[] {TSDataType.FLOAT, TSDataType.DOUBLE, TSDataType.INT32};
-
- String[] columns = new String[3];
- columns[0] = 2.0 + "";
- columns[1] = 10000 + "";
- columns[2] = 100 + "";
-
- InsertRowPlan insertRowPlan =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.aligned_device"),
- time,
- new String[] {"s1", "s2", "s3"},
- dataTypes,
- columns,
- true);
- insertRowPlan.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- // call getSeriesSchemasAndReadLockDevice
- IMNode node = schemaProcessor.getSeriesSchemasAndReadLockDevice(insertRowPlan);
- assertEquals(
- 3, schemaProcessor.getAllTimeseriesCount(node.getPartialPath().concatNode("**")));
-
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- @Test
- public void testCreateAlignedTimeseriesAndInsertWithNotAlignedData() {
- LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
- try {
- schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
- schemaProcessor.createAlignedTimeSeries(
- new PartialPath("root.laptop.d1.aligned_device"),
- Arrays.asList("s1", "s2", "s3"),
- Arrays.asList(
- TSDataType.valueOf("FLOAT"),
- TSDataType.valueOf("INT64"),
- TSDataType.valueOf("INT32")),
- Arrays.asList(
- TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE")),
- Arrays.asList(compressionType, compressionType, compressionType));
- } catch (Exception e) {
- fail();
- }
-
- try {
- schemaProcessor.createTimeseries(
- new CreateTimeSeriesPlan(
- new PartialPath("root.laptop.d1.aligned_device.s4"),
- TSDataType.valueOf("FLOAT"),
- TSEncoding.valueOf("RLE"),
- compressionType,
- null,
- null,
- null,
- null));
- fail();
- } catch (Exception e) {
- Assert.assertEquals(
- "Timeseries under this entity is aligned, please use createAlignedTimeseries or change entity. (Path: root.laptop.d1.aligned_device)",
- e.getMessage());
- }
-
- try {
- // construct an insertRowPlan with mismatched data type
- long time = 1L;
- TSDataType[] dataTypes =
- new TSDataType[] {TSDataType.FLOAT, TSDataType.INT64, TSDataType.INT32};
-
- String[] columns = new String[3];
- columns[0] = "1.0";
- columns[1] = "2";
- columns[2] = "3";
-
- InsertRowPlan insertRowPlan =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.aligned_device"),
- time,
- new String[] {"s1", "s2", "s3"},
- dataTypes,
- columns,
- false);
- insertRowPlan.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- // call getSeriesSchemasAndReadLockDevice
- schemaProcessor.getSeriesSchemasAndReadLockDevice(insertRowPlan);
- fail();
- } catch (Exception e) {
- Assert.assertEquals(
- "timeseries under this device are aligned, please use aligned interface (Path: root.laptop.d1.aligned_device)",
- e.getMessage());
- }
- }
-
- @Test
- public void testCreateTimeseriesAndInsertWithMismatchDataType() {
- LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
- try {
- schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
- schemaProcessor.createTimeseries(
- new PartialPath("root.laptop.d1.s0"),
- TSDataType.valueOf("INT32"),
- TSEncoding.valueOf("RLE"),
- compressionType,
- Collections.emptyMap());
-
- // construct an insertRowPlan with mismatched data type
- long time = 1L;
- TSDataType[] dataTypes = new TSDataType[] {TSDataType.FLOAT};
-
- String[] columns = new String[1];
- columns[0] = 2.0 + "";
-
- InsertRowPlan insertRowPlan =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1"), time, new String[] {"s0"}, dataTypes, columns);
- insertRowPlan.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- // call getSeriesSchemasAndReadLockDevice
- IMNode node = schemaProcessor.getSeriesSchemasAndReadLockDevice(insertRowPlan);
- assertEquals(
- 1, schemaProcessor.getAllTimeseriesCount(node.getPartialPath().concatNode("**")));
- assertNull(insertRowPlan.getMeasurementMNodes()[0]);
- assertEquals(1, insertRowPlan.getFailedMeasurementNumber());
-
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
@Test
public void testCreateTimeseriesAndInsertWithAlignedData() {
LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
@@ -2150,35 +1993,6 @@ public abstract class SchemaBasicTest {
"Timeseries under this entity is not aligned, please use createTimeseries or change entity. (Path: root.laptop.d1.aligned_device)",
e.getMessage());
}
-
- try {
- // construct an insertRowPlan with mismatched data type
- long time = 1L;
- TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
-
- String[] columns = new String[2];
- columns[0] = "1";
- columns[1] = "2";
-
- InsertRowPlan insertRowPlan =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.aligned_device"),
- time,
- new String[] {"s1", "s2"},
- dataTypes,
- columns,
- true);
- insertRowPlan.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- // call getSeriesSchemasAndReadLockDevice
- schemaProcessor.getSeriesSchemasAndReadLockDevice(insertRowPlan);
- fail();
- } catch (Exception e) {
- Assert.assertEquals(
- "timeseries under this device are not aligned, please use non-aligned interface (Path: root.laptop.d1.aligned_device)",
- e.getMessage());
- }
}
@Test
@@ -2300,51 +2114,6 @@ public abstract class SchemaBasicTest {
Assert.assertEquals(tags, result.getTag());
}
- @Test
- public void testAutoCreateAlignedTimeseriesWhileInsert() {
- LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-
- try {
- long time = 1L;
- TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT32};
-
- String[] columns = new String[2];
- columns[0] = "1";
- columns[1] = "2";
-
- InsertRowPlan insertRowPlan =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.aligned_device"),
- time,
- new String[] {"s1", "s2"},
- dataTypes,
- columns,
- true);
- insertRowPlan.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- schemaProcessor.getSeriesSchemasAndReadLockDevice(insertRowPlan);
-
- assertTrue(schemaProcessor.isPathExist(new PartialPath("root.laptop.d1.aligned_device.s1")));
- assertTrue(schemaProcessor.isPathExist(new PartialPath("root.laptop.d1.aligned_device.s2")));
-
- insertRowPlan.setMeasurements(new String[] {"s3", "s4"});
- schemaProcessor.getSeriesSchemasAndReadLockDevice(insertRowPlan);
- assertTrue(schemaProcessor.isPathExist(new PartialPath("root.laptop.d1.aligned_device.s3")));
- assertTrue(schemaProcessor.isPathExist(new PartialPath("root.laptop.d1.aligned_device.s4")));
-
- insertRowPlan.setMeasurements(new String[] {"s2", "s5"});
- schemaProcessor.getSeriesSchemasAndReadLockDevice(insertRowPlan);
- assertTrue(schemaProcessor.isPathExist(new PartialPath("root.laptop.d1.aligned_device.s5")));
-
- insertRowPlan.setMeasurements(new String[] {"s2", "s3"});
- schemaProcessor.getSeriesSchemasAndReadLockDevice(insertRowPlan);
-
- } catch (MetadataException | IOException e) {
- fail();
- }
- }
-
@Test
public void testGetStorageGroupNodeByPath() {
LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
@@ -2382,48 +2151,6 @@ public abstract class SchemaBasicTest {
}
}
- @Test
- public void testMeasurementIdWhileInsert() throws Exception {
- LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-
- PartialPath deviceId = new PartialPath("root.sg.d");
- InsertPlan insertPlan;
-
- insertPlan = getInsertPlan("`\"a+b\"`");
- schemaProcessor.getSeriesSchemasAndReadLockDevice(insertPlan);
- assertTrue(schemaProcessor.isPathExist(deviceId.concatNode("`\"a+b\"`")));
-
- insertPlan = getInsertPlan("`\"a.b\"`");
- schemaProcessor.getSeriesSchemasAndReadLockDevice(insertPlan);
- assertTrue(schemaProcessor.isPathExist(deviceId.concatNode("`\"a.b\"`")));
-
- insertPlan = getInsertPlan("`\"a“(Φ)”b\"`");
- schemaProcessor.getSeriesSchemasAndReadLockDevice(insertPlan);
- assertTrue(schemaProcessor.isPathExist(deviceId.concatNode("`\"a“(Φ)”b\"`")));
-
- String[] illegalMeasurementIds = {"time", "timestamp", "TIME", "TIMESTAMP"};
- for (String measurementId : illegalMeasurementIds) {
- insertPlan = getInsertPlan(measurementId);
- try {
- schemaProcessor.getSeriesSchemasAndReadLockDevice(insertPlan);
- assertFalse(schemaProcessor.isPathExist(deviceId.concatNode(measurementId)));
- } catch (MetadataException e) {
- e.printStackTrace();
- }
- }
- }
-
- private InsertPlan getInsertPlan(String measurementId) throws MetadataException {
- PartialPath deviceId = new PartialPath("root.sg.d");
- String[] measurementList = {measurementId};
- String[] values = {"1"};
- IMeasurementMNode[] measurementMNodes = new IMeasurementMNode[1];
- InsertPlan insertPlan = new InsertRowPlan(deviceId, 1L, measurementList, values);
- insertPlan.setMeasurementMNodes(measurementMNodes);
- insertPlan.getDataTypes()[0] = TSDataType.INT32;
- return insertPlan;
- }
-
@Test
public void testTemplateSchemaNameCheckWhileCreate() {
LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
@@ -2458,31 +2185,6 @@ public abstract class SchemaBasicTest {
"template1", schemaNames, measurementList, dataTypeList, encodingList, compressionTypes);
}
- @Test
- public void testDeviceNodeAfterAutoCreateTimeseriesFailure() throws Exception {
- LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-
- PartialPath sg1 = new PartialPath("root.a.sg");
- schemaProcessor.setStorageGroup(sg1);
-
- PartialPath deviceId = new PartialPath("root.a.d");
- String[] measurementList = {"s"};
- String[] values = {"1"};
- IMeasurementMNode[] measurementMNodes = new IMeasurementMNode[1];
- InsertPlan insertPlan = new InsertRowPlan(deviceId, 1L, measurementList, values);
- insertPlan.setMeasurementMNodes(measurementMNodes);
- insertPlan.getDataTypes()[0] = TSDataType.INT32;
-
- try {
- schemaProcessor.getSeriesSchemasAndReadLockDevice(insertPlan);
- fail();
- } catch (MetadataException e) {
- Assert.assertEquals(
- "some children of root.a have already been set to storage group", e.getMessage());
- Assert.assertFalse(schemaProcessor.isPathExist(new PartialPath("root.a.d")));
- }
- }
-
@Test
@Ignore
public void testTimeseriesDeletionWithEntityUsingTemplate() throws MetadataException {
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/TemplateTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/TemplateTest.java
index 819d8cdedb..41d8aff180 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/TemplateTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/TemplateTest.java
@@ -18,7 +18,6 @@
*/
package org.apache.iotdb.db.metadata;
-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;
@@ -27,7 +26,6 @@ import org.apache.iotdb.db.metadata.mnode.IMNode;
import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
import org.apache.iotdb.db.metadata.template.Template;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
import org.apache.iotdb.db.qp.physical.sys.CreateTemplatePlan;
import org.apache.iotdb.db.qp.physical.sys.DropTemplatePlan;
@@ -476,16 +474,4 @@ public class TemplateTest {
e.getMessage());
}
}
-
- private InsertRowPlan getInsertRowPlan(String prefixPath, String measurement)
- throws IllegalPathException {
- long time = 110L;
- TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT64};
-
- String[] columns = new String[1];
- columns[0] = "1";
-
- return new InsertRowPlan(
- new PartialPath(prefixPath), time, new String[] {measurement}, dataTypes, columns);
- }
}
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableLogFileTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableLogFileTest.java
index eb2b337518..cccadde68a 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableLogFileTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableLogFileTest.java
@@ -19,27 +19,6 @@
package org.apache.iotdb.db.metadata.idtable;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.metadata.idtable.entry.DeviceIDFactory;
-import org.apache.iotdb.db.metadata.idtable.entry.DiskSchemaEntry;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
public class IDTableLogFileTest {
private boolean isEnableIDTable = false;
@@ -48,113 +27,113 @@ public class IDTableLogFileTest {
private boolean isEnableIDTableLogFile = false;
- @Before
- public void before() {
- IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(true);
- isEnableIDTable = IoTDBDescriptor.getInstance().getConfig().isEnableIDTable();
- originalDeviceIDTransformationMethod =
- IoTDBDescriptor.getInstance().getConfig().getDeviceIDTransformationMethod();
- isEnableIDTableLogFile = IoTDBDescriptor.getInstance().getConfig().isEnableIDTableLogFile();
-
- IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(true);
- IoTDBDescriptor.getInstance().getConfig().setDeviceIDTransformationMethod("SHA256");
- IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(true);
- EnvironmentUtils.envSetUp();
- }
-
- @After
- public void clean() throws IOException, StorageEngineException {
- IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(isEnableIDTable);
- IoTDBDescriptor.getInstance()
- .getConfig()
- .setDeviceIDTransformationMethod(originalDeviceIDTransformationMethod);
- IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(isEnableIDTableLogFile);
- EnvironmentUtils.cleanEnv();
- }
-
- @Test
- public void testInsertAndAutoCreate() {
- try {
- // construct an insertRowPlan with mismatched data type
- long time = 1L;
- TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
-
- String[] columns = new String[2];
- columns[0] = "1";
- columns[1] = "2";
-
- InsertRowPlan insertRowPlan =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.non_aligned_device1"),
- time,
- new String[] {"s1", "s2"},
- dataTypes,
- columns,
- false);
- insertRowPlan.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- // call getSeriesSchemasAndReadLockDevice
- IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
- idTable.getSeriesSchemas(insertRowPlan);
-
- insertRowPlan =
- new InsertRowPlan(
- new PartialPath("root.laptop.d2.non_aligned_device2"),
- time,
- new String[] {"s3", "s4"},
- dataTypes,
- columns,
- false);
- insertRowPlan.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- idTable.getSeriesSchemas(insertRowPlan);
-
- // test part
- HashMap<String, HashSet<String>> deviceIdToNameMap = new HashMap<>();
- HashSet<String> d1Set = new HashSet<>();
- d1Set.add("root.laptop.d1.non_aligned_device1.s1");
- d1Set.add("root.laptop.d1.non_aligned_device1.s2");
- HashSet<String> d2Set = new HashSet<>();
- d2Set.add("root.laptop.d2.non_aligned_device2.s3");
- d2Set.add("root.laptop.d2.non_aligned_device2.s4");
-
- HashSet<String> resSet = new HashSet<>();
-
- deviceIdToNameMap.put(
- DeviceIDFactory.getInstance()
- .getDeviceID(new PartialPath("root.laptop.d1.non_aligned_device1"))
- .toStringID(),
- d1Set);
- deviceIdToNameMap.put(
- DeviceIDFactory.getInstance()
- .getDeviceID(new PartialPath("root.laptop.d2.non_aligned_device2"))
- .toStringID(),
- d2Set);
-
- int count = 0;
- for (DiskSchemaEntry entry : idTable.getIDiskSchemaManager().getAllSchemaEntry()) {
- String deviceID = entry.deviceID;
- HashSet<String> set = deviceIdToNameMap.get(deviceID);
-
- if (set == null) {
- fail("device path is not correct " + deviceID);
- }
-
- if (!set.contains(entry.seriesKey)) {
- fail("series path is not correct " + entry.seriesKey);
- }
- count++;
- resSet.add(entry.seriesKey);
- }
-
- assertEquals(4, count);
- assertEquals(4, resSet.size());
- } catch (Exception e) {
- e.printStackTrace();
- fail("throw exception");
- }
- }
+ // @Before
+ // public void before() {
+ // IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(true);
+ // isEnableIDTable = IoTDBDescriptor.getInstance().getConfig().isEnableIDTable();
+ // originalDeviceIDTransformationMethod =
+ // IoTDBDescriptor.getInstance().getConfig().getDeviceIDTransformationMethod();
+ // isEnableIDTableLogFile = IoTDBDescriptor.getInstance().getConfig().isEnableIDTableLogFile();
+ //
+ // IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(true);
+ // IoTDBDescriptor.getInstance().getConfig().setDeviceIDTransformationMethod("SHA256");
+ // IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(true);
+ // EnvironmentUtils.envSetUp();
+ // }
+ //
+ // @After
+ // public void clean() throws IOException, StorageEngineException {
+ // IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(isEnableIDTable);
+ // IoTDBDescriptor.getInstance()
+ // .getConfig()
+ // .setDeviceIDTransformationMethod(originalDeviceIDTransformationMethod);
+ // IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(isEnableIDTableLogFile);
+ // EnvironmentUtils.cleanEnv();
+ // }
+
+ // @Test
+ // public void testInsertAndAutoCreate() {
+ // try {
+ // // construct an insertRowPlan with mismatched data type
+ // long time = 1L;
+ // TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
+ //
+ // String[] columns = new String[2];
+ // columns[0] = "1";
+ // columns[1] = "2";
+ //
+ // InsertRowPlan insertRowPlan =
+ // new InsertRowPlan(
+ // new PartialPath("root.laptop.d1.non_aligned_device1"),
+ // time,
+ // new String[] {"s1", "s2"},
+ // dataTypes,
+ // columns,
+ // false);
+ // insertRowPlan.setMeasurementMNodes(
+ // new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+ //
+ // // call getSeriesSchemasAndReadLockDevice
+ // IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+ //
+ // idTable.getSeriesSchemas(insertRowPlan);
+ //
+ // insertRowPlan =
+ // new InsertRowPlan(
+ // new PartialPath("root.laptop.d2.non_aligned_device2"),
+ // time,
+ // new String[] {"s3", "s4"},
+ // dataTypes,
+ // columns,
+ // false);
+ // insertRowPlan.setMeasurementMNodes(
+ // new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+ //
+ // idTable.getSeriesSchemas(insertRowPlan);
+ //
+ // // test part
+ // HashMap<String, HashSet<String>> deviceIdToNameMap = new HashMap<>();
+ // HashSet<String> d1Set = new HashSet<>();
+ // d1Set.add("root.laptop.d1.non_aligned_device1.s1");
+ // d1Set.add("root.laptop.d1.non_aligned_device1.s2");
+ // HashSet<String> d2Set = new HashSet<>();
+ // d2Set.add("root.laptop.d2.non_aligned_device2.s3");
+ // d2Set.add("root.laptop.d2.non_aligned_device2.s4");
+ //
+ // HashSet<String> resSet = new HashSet<>();
+ //
+ // deviceIdToNameMap.put(
+ // DeviceIDFactory.getInstance()
+ // .getDeviceID(new PartialPath("root.laptop.d1.non_aligned_device1"))
+ // .toStringID(),
+ // d1Set);
+ // deviceIdToNameMap.put(
+ // DeviceIDFactory.getInstance()
+ // .getDeviceID(new PartialPath("root.laptop.d2.non_aligned_device2"))
+ // .toStringID(),
+ // d2Set);
+ //
+ // int count = 0;
+ // for (DiskSchemaEntry entry : idTable.getIDiskSchemaManager().getAllSchemaEntry()) {
+ // String deviceID = entry.deviceID;
+ // HashSet<String> set = deviceIdToNameMap.get(deviceID);
+ //
+ // if (set == null) {
+ // fail("device path is not correct " + deviceID);
+ // }
+ //
+ // if (!set.contains(entry.seriesKey)) {
+ // fail("series path is not correct " + entry.seriesKey);
+ // }
+ // count++;
+ // resSet.add(entry.seriesKey);
+ // }
+ //
+ // assertEquals(4, count);
+ // assertEquals(4, resSet.size());
+ // } catch (Exception e) {
+ // e.printStackTrace();
+ // fail("throw exception");
+ // }
+ // }
}
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableTest.java
index 2ea9f8cd35..836fc89092 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableTest.java
@@ -18,737 +18,748 @@
*/
package org.apache.iotdb.db.metadata.idtable;
-
-import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.trigger.service.TriggerRegistrationService;
-import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.metadata.LocalSchemaProcessor;
-import org.apache.iotdb.db.metadata.idtable.entry.DeviceIDFactory;
-import org.apache.iotdb.db.metadata.idtable.entry.DiskSchemaEntry;
-import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
-import org.apache.iotdb.db.metadata.idtable.entry.SchemaEntry;
-import org.apache.iotdb.db.metadata.lastCache.container.ILastCacheContainer;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.qp.Planner;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
-import org.apache.iotdb.db.qp.physical.sys.CreateTriggerPlan;
-import org.apache.iotdb.db.qp.physical.sys.DropTriggerPlan;
-import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
-import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.tsfile.read.TimeValuePair;
-import org.apache.iotdb.tsfile.utils.Pair;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class IDTableTest {
-
- private CompressionType compressionType;
-
- private boolean isEnableIDTable = false;
-
- private String originalDeviceIDTransformationMethod = null;
-
- private boolean isEnableIDTableLogFile = false;
-
- @Before
- public void before() {
- compressionType = TSFileDescriptor.getInstance().getConfig().getCompressor();
- IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(true);
- isEnableIDTable = IoTDBDescriptor.getInstance().getConfig().isEnableIDTable();
- originalDeviceIDTransformationMethod =
- IoTDBDescriptor.getInstance().getConfig().getDeviceIDTransformationMethod();
- isEnableIDTableLogFile = IoTDBDescriptor.getInstance().getConfig().isEnableIDTableLogFile();
-
- IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(true);
- IoTDBDescriptor.getInstance().getConfig().setDeviceIDTransformationMethod("SHA256");
- IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(true);
- EnvironmentUtils.envSetUp();
- }
-
- @After
- public void clean() throws IOException, StorageEngineException {
- IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(isEnableIDTable);
- IoTDBDescriptor.getInstance()
- .getConfig()
- .setDeviceIDTransformationMethod(originalDeviceIDTransformationMethod);
- IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(isEnableIDTableLogFile);
- EnvironmentUtils.cleanEnv();
- }
-
- @Test
- public void testCreateAlignedTimeseriesAndInsert() {
- LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-
- try {
- schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
- CreateAlignedTimeSeriesPlan plan =
- new CreateAlignedTimeSeriesPlan(
- new PartialPath("root.laptop.d1.aligned_device"),
- Arrays.asList("s1", "s2", "s3"),
- Arrays.asList(
- TSDataType.valueOf("FLOAT"),
- TSDataType.valueOf("INT64"),
- TSDataType.valueOf("INT32")),
- Arrays.asList(
- TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE")),
- Arrays.asList(compressionType, compressionType, compressionType),
- null,
- null,
- null);
-
- schemaProcessor.createAlignedTimeSeries(plan);
-
- IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
- // construct an insertRowPlan with mismatched data type
- long time = 1L;
- TSDataType[] dataTypes =
- new TSDataType[] {TSDataType.FLOAT, TSDataType.INT64, TSDataType.INT32};
-
- String[] columns = new String[3];
- columns[0] = 2.0 + "";
- columns[1] = 10000 + "";
- columns[2] = 100 + "";
-
- InsertRowPlan insertRowPlan =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.aligned_device"),
- time,
- new String[] {"s1", "s2", "s3"},
- dataTypes,
- columns,
- true);
- insertRowPlan.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- idTable.getSeriesSchemas(insertRowPlan);
-
- // with type mismatch
- dataTypes = new TSDataType[] {TSDataType.FLOAT, TSDataType.DOUBLE, TSDataType.INT32};
- InsertRowPlan insertRowPlan2 =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.aligned_device"),
- time,
- new String[] {"s1", "s2", "s3"},
- dataTypes,
- columns,
- true);
- insertRowPlan2.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- // we should throw type mismatch exception here
- try {
- IoTDBDescriptor.getInstance().getConfig().setEnablePartialInsert(false);
- idTable.getSeriesSchemas(insertRowPlan2);
- fail("should throw exception");
- } catch (DataTypeMismatchException e) {
- assertEquals(
- "data type of root.laptop.d1.aligned_device.s2 is not consistent, registered type INT64, inserting type DOUBLE, timestamp 1, value 10000.0",
- e.getMessage());
- } catch (Exception e2) {
- fail("throw wrong exception");
- }
-
- IoTDBDescriptor.getInstance().getConfig().setEnablePartialInsert(true);
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- @Test
- public void testCreateAlignedTimeseriesAndInsertNotAlignedData() {
- LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-
- try {
- schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
- CreateAlignedTimeSeriesPlan plan =
- new CreateAlignedTimeSeriesPlan(
- new PartialPath("root.laptop.d1.aligned_device"),
- Arrays.asList("s1", "s2", "s3"),
- Arrays.asList(
- TSDataType.valueOf("FLOAT"),
- TSDataType.valueOf("INT64"),
- TSDataType.valueOf("INT32")),
- Arrays.asList(
- TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE")),
- Arrays.asList(compressionType, compressionType, compressionType),
- null,
- null,
- null);
-
- schemaProcessor.createAlignedTimeSeries(plan);
-
- IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
- // construct an insertRowPlan with mismatched data type
- long time = 1L;
- TSDataType[] dataTypes =
- new TSDataType[] {TSDataType.FLOAT, TSDataType.INT64, TSDataType.INT32};
-
- String[] columns = new String[3];
- columns[0] = 2.0 + "";
- columns[1] = 10000 + "";
- columns[2] = 100 + "";
-
- // non aligned plan
- InsertRowPlan insertRowPlan =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.aligned_device"),
- time,
- new String[] {"s1", "s2", "s3"},
- dataTypes,
- columns,
- false);
- insertRowPlan.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- // call getSeriesSchemasAndReadLockDevice
- try {
- idTable.getSeriesSchemas(insertRowPlan);
- fail("should throw exception");
- } catch (MetadataException e) {
- assertEquals(
- "Timeseries under path [root.laptop.d1.aligned_device]'s align value is [true], which is not consistent with insert plan",
- e.getMessage());
- } catch (Exception e2) {
- fail("throw wrong exception");
- }
-
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- @Test
- public void testCreateTimeseriesAndInsert() {
- LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
- try {
- schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
- schemaProcessor.createTimeseries(
- new PartialPath("root.laptop.d1.s0"),
- TSDataType.valueOf("INT32"),
- TSEncoding.valueOf("RLE"),
- compressionType,
- Collections.emptyMap());
-
- IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
- long time = 1L;
- String[] columns = new String[1];
- columns[0] = 2 + "";
-
- // correct insert plan
- InsertRowPlan insertRowPlan =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1"),
- time,
- new String[] {"s0"},
- new TSDataType[] {TSDataType.INT32},
- columns);
- insertRowPlan.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- idTable.getSeriesSchemas(insertRowPlan);
- assertEquals(insertRowPlan.getMeasurementMNodes()[0].getSchema().getType(), TSDataType.INT32);
- assertEquals(0, insertRowPlan.getFailedMeasurementNumber());
-
- // construct an insertRowPlan with mismatched data type
- InsertRowPlan insertRowPlan2 =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1"),
- time,
- new String[] {"s0"},
- new TSDataType[] {TSDataType.FLOAT},
- columns);
- insertRowPlan2.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- // get series schema
- idTable.getSeriesSchemas(insertRowPlan2);
- assertNull(insertRowPlan2.getMeasurementMNodes()[0]);
- assertEquals(1, insertRowPlan2.getFailedMeasurementNumber());
-
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- @Test
- public void testCreateTimeseriesAndInsertWithAlignedData() {
- LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
- try {
- schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
- schemaProcessor.createTimeseries(
- new PartialPath("root.laptop.d1.non_aligned_device.s1"),
- TSDataType.valueOf("INT32"),
- TSEncoding.valueOf("RLE"),
- compressionType,
- Collections.emptyMap());
- schemaProcessor.createTimeseries(
- new PartialPath("root.laptop.d1.non_aligned_device.s2"),
- TSDataType.valueOf("INT64"),
- TSEncoding.valueOf("RLE"),
- compressionType,
- Collections.emptyMap());
-
- // construct an insertRowPlan with mismatched data type
- long time = 1L;
- TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
-
- String[] columns = new String[2];
- columns[0] = "1";
- columns[1] = "2";
-
- InsertRowPlan insertRowPlan =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.non_aligned_device"),
- time,
- new String[] {"s1", "s2"},
- dataTypes,
- columns,
- true);
- insertRowPlan.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- // call getSeriesSchemasAndReadLockDevice
- IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
- try {
- idTable.getSeriesSchemas(insertRowPlan);
- fail("should throw exception");
- } catch (MetadataException e) {
- assertEquals(
- "Timeseries under path [root.laptop.d1.non_aligned_device]'s align value is [false], which is not consistent with insert plan",
- e.getMessage());
- }
- } catch (Exception e) {
- fail("throw wrong exception");
- }
- }
-
- @Test
- public void testInsertAndAutoCreate() {
- LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
- try {
- // construct an insertRowPlan with mismatched data type
- long time = 1L;
- TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
-
- String[] columns = new String[2];
- columns[0] = "1";
- columns[1] = "2";
-
- InsertRowPlan insertRowPlan =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.non_aligned_device"),
- time,
- new String[] {"s1", "s2"},
- dataTypes,
- columns,
- false);
- insertRowPlan.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- // call getSeriesSchemasAndReadLockDevice
- IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
- idTable.getSeriesSchemas(insertRowPlan);
-
- // check SchemaProcessor
- IMeasurementMNode s1Node =
- schemaProcessor.getMeasurementMNode(
- new PartialPath("root.laptop.d1.non_aligned_device.s1"));
- assertEquals("s1", s1Node.getName());
- assertEquals(TSDataType.INT32, s1Node.getSchema().getType());
- IMeasurementMNode s2Node =
- schemaProcessor.getMeasurementMNode(
- new PartialPath("root.laptop.d1.non_aligned_device.s2"));
- assertEquals("s2", s2Node.getName());
- assertEquals(TSDataType.INT64, s2Node.getSchema().getType());
-
- // insert type mismatch data
- InsertRowPlan insertRowPlan2 =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.non_aligned_device"),
- time,
- new String[] {"s1", "s2"},
- new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
- columns,
- false);
- insertRowPlan2.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- idTable.getSeriesSchemas(insertRowPlan2);
-
- assertNull(insertRowPlan2.getMeasurementMNodes()[0]);
- assertEquals(insertRowPlan.getMeasurementMNodes()[1].getSchema().getType(), TSDataType.INT64);
- assertEquals(1, insertRowPlan2.getFailedMeasurementNumber());
-
- // insert aligned data
- InsertRowPlan insertRowPlan3 =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.non_aligned_device"),
- time,
- new String[] {"s1", "s2"},
- new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
- columns,
- true);
- insertRowPlan3.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- try {
- idTable.getSeriesSchemas(insertRowPlan3);
- fail("should throw exception");
- } catch (MetadataException e) {
- assertEquals(
- "Timeseries under path [root.laptop.d1.non_aligned_device]'s align value is [false], which is not consistent with insert plan",
- e.getMessage());
- } catch (Exception e) {
- fail("throw wrong exception");
- }
- } catch (MetadataException e) {
- e.printStackTrace();
- fail("throw exception");
- }
- }
-
- @Test
- public void testAlignedInsertAndAutoCreate() {
- LocalSchemaProcessor processor = IoTDB.schemaProcessor;
- try {
- // construct an insertRowPlan with mismatched data type
- long time = 1L;
- TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
-
- String[] columns = new String[2];
- columns[0] = "1";
- columns[1] = "2";
-
- InsertRowPlan insertRowPlan =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.aligned_device"),
- time,
- new String[] {"s1", "s2"},
- dataTypes,
- columns,
- true);
- insertRowPlan.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- // call getSeriesSchemasAndReadLockDevice
- IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
- idTable.getSeriesSchemas(insertRowPlan);
-
- // check SchemaProcessor
- IMeasurementMNode s1Node =
- processor.getMeasurementMNode(new PartialPath("root.laptop.d1.aligned_device.s1"));
- assertEquals("s1", s1Node.getName());
- assertEquals(TSDataType.INT32, s1Node.getSchema().getType());
- IMeasurementMNode s2Node =
- processor.getMeasurementMNode(new PartialPath("root.laptop.d1.aligned_device.s2"));
- assertEquals("s2", s2Node.getName());
- assertEquals(TSDataType.INT64, s2Node.getSchema().getType());
- assertTrue(s2Node.getParent().isAligned());
-
- // insert type mismatch data
- InsertRowPlan insertRowPlan2 =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.aligned_device"),
- time,
- new String[] {"s1", "s2"},
- new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
- columns,
- true);
- insertRowPlan2.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- idTable.getSeriesSchemas(insertRowPlan2);
-
- assertNull(insertRowPlan2.getMeasurementMNodes()[0]);
- assertEquals(insertRowPlan.getMeasurementMNodes()[1].getSchema().getType(), TSDataType.INT64);
- assertEquals(1, insertRowPlan2.getFailedMeasurementNumber());
-
- // insert non-aligned data
- InsertRowPlan insertRowPlan3 =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.aligned_device"),
- time,
- new String[] {"s1", "s2"},
- new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
- columns,
- false);
- insertRowPlan3.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- try {
- idTable.getSeriesSchemas(insertRowPlan3);
- fail("should throw exception");
- } catch (MetadataException e) {
- assertEquals(
- "Timeseries under path [root.laptop.d1.aligned_device]'s align value is [true], which is not consistent with insert plan",
- e.getMessage());
- } catch (Exception e) {
- fail("throw wrong exception");
- }
- } catch (MetadataException e) {
- e.printStackTrace();
- fail("throw exception");
- }
- }
-
- @Test
- public void testTriggerAndInsert() {
- LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
- try {
- long time = 1L;
-
- schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
- schemaProcessor.createTimeseries(
- new PartialPath("root.laptop.d1.non_aligned_device.s1"),
- TSDataType.valueOf("INT32"),
- TSEncoding.valueOf("RLE"),
- compressionType,
- Collections.emptyMap());
- schemaProcessor.createTimeseries(
- new PartialPath("root.laptop.d1.non_aligned_device.s2"),
- TSDataType.valueOf("INT64"),
- TSEncoding.valueOf("RLE"),
- compressionType,
- Collections.emptyMap());
-
- Planner processor = new Planner();
-
- String sql =
- "CREATE TRIGGER trigger1 BEFORE INSERT ON root.laptop.d1.non_aligned_device.s1 AS 'org.apache.iotdb.db.metadata.idtable.trigger_example.Counter'";
-
- CreateTriggerPlan plan = (CreateTriggerPlan) processor.parseSQLToPhysicalPlan(sql);
-
- TriggerRegistrationService.getInstance().register(plan);
-
- TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
- String[] columns = new String[2];
- columns[0] = "1";
- columns[1] = "2";
-
- InsertRowPlan insertRowPlan =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.non_aligned_device"),
- time,
- new String[] {"s1", "s2"},
- dataTypes,
- columns,
- false);
- insertRowPlan.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- // call getSeriesSchemasAndReadLockDevice
- IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
- idTable.getSeriesSchemas(insertRowPlan);
-
- // check SchemaProcessor
- IMeasurementMNode s1Node =
- schemaProcessor.getMeasurementMNode(
- new PartialPath("root.laptop.d1.non_aligned_device.s1"));
- assertEquals("s1", s1Node.getName());
- assertEquals(TSDataType.INT32, s1Node.getSchema().getType());
- assertNotNull(s1Node.getTriggerExecutor());
-
- IMeasurementMNode s2Node =
- schemaProcessor.getMeasurementMNode(
- new PartialPath("root.laptop.d1.non_aligned_device.s2"));
- assertEquals("s2", s2Node.getName());
- assertEquals(TSDataType.INT64, s2Node.getSchema().getType());
- assertNull(s2Node.getTriggerExecutor());
-
- // drop trigger
- String sql2 = "Drop trigger trigger1";
-
- DropTriggerPlan plan2 = (DropTriggerPlan) processor.parseSQLToPhysicalPlan(sql2);
- TriggerRegistrationService.getInstance().deregister(plan2);
-
- idTable.getSeriesSchemas(insertRowPlan);
- assertNull(s1Node.getTriggerExecutor());
- } catch (MetadataException | StorageEngineException | QueryProcessException e) {
- e.printStackTrace();
- fail("throw exception");
- }
- }
-
- @Test
- public void testGetDiskSchemaEntries() {
- try {
- IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
- String sgPath = "root.laptop";
- for (int i = 0; i < 10; i++) {
- String devicePath = sgPath + ".d" + i;
- IDeviceID iDeviceID = DeviceIDFactory.getInstance().getDeviceID(devicePath);
- String measurement = "s" + i;
- idTable.putSchemaEntry(
- devicePath,
- measurement,
- new SchemaEntry(
- TSDataType.BOOLEAN,
- TSEncoding.BITMAP,
- CompressionType.UNCOMPRESSED,
- iDeviceID,
- new PartialPath(devicePath + "." + measurement),
- false,
- idTable.getIDiskSchemaManager()),
- false);
- SchemaEntry schemaEntry =
- idTable.getDeviceEntry(iDeviceID.toStringID()).getSchemaEntry(measurement);
- List<SchemaEntry> schemaEntries = new ArrayList<>();
- schemaEntries.add(schemaEntry);
- List<DiskSchemaEntry> diskSchemaEntries = idTable.getDiskSchemaEntries(schemaEntries);
- assertNotNull(diskSchemaEntries);
- assertEquals(diskSchemaEntries.size(), 1);
- assertEquals(diskSchemaEntries.get(0).seriesKey, devicePath + "." + measurement);
- }
- } catch (Exception e) {
- e.printStackTrace();
- fail("throw exception");
- }
- }
-
- @Test
- public void testDeleteTimeseries() {
- try {
- IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
- String sgPath = "root.laptop";
- for (int i = 0; i < 10; i++) {
- String devicePath = sgPath + ".d" + i;
- IDeviceID iDeviceID = DeviceIDFactory.getInstance().getDeviceID(devicePath);
- String measurement = "s" + i;
- SchemaEntry schemaEntry =
- new SchemaEntry(
- TSDataType.BOOLEAN,
- TSEncoding.BITMAP,
- CompressionType.UNCOMPRESSED,
- iDeviceID,
- new PartialPath(devicePath + "." + measurement),
- false,
- idTable.getIDiskSchemaManager());
- idTable.putSchemaEntry(devicePath, measurement, schemaEntry, false);
- }
- List<PartialPath> partialPaths = new ArrayList<>();
- partialPaths.add(new PartialPath("root.laptop.d0.s0"));
- partialPaths.add(new PartialPath("root.laptop.d8.s8"));
- partialPaths.add(new PartialPath("root.laptop.d2.s3"));
- Pair<Integer, Set<String>> pairs = idTable.deleteTimeseries(partialPaths);
- assertNotNull(pairs);
- assertEquals((int) pairs.left, 2);
- assertTrue(pairs.right.contains("root.laptop.d2.s3"));
- assertFalse(pairs.right.contains("root.laptop.d0.s0"));
- assertFalse(pairs.right.contains("root.laptop.d8.s8"));
- Collection<DiskSchemaEntry> diskSchemaEntries =
- idTable.getIDiskSchemaManager().getAllSchemaEntry();
- for (DiskSchemaEntry diskSchemaEntry : diskSchemaEntries) {
- assertNotEquals("root.laptop.d0.s0", diskSchemaEntry.seriesKey);
- assertNotEquals("root.laptop.d8.s8", diskSchemaEntry.seriesKey);
- }
- assertNull(idTable.getDeviceEntry("root.laptop.d0").getMeasurementMap().get("s0"));
- assertNull(idTable.getDeviceEntry("root.laptop.d8").getMeasurementMap().get("s1"));
- } catch (Exception e) {
- e.printStackTrace();
- fail("throw exception");
- }
- }
-
- @Test
- public void testFlushTimeAndLastCache() {
- LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
- try {
- long time = 1L;
-
- schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
- schemaProcessor.createTimeseries(
- new PartialPath("root.laptop.d1.non_aligned_device.s1"),
- TSDataType.valueOf("INT32"),
- TSEncoding.valueOf("RLE"),
- compressionType,
- Collections.emptyMap());
- schemaProcessor.createTimeseries(
- new PartialPath("root.laptop.d1.non_aligned_device.s2"),
- TSDataType.valueOf("INT64"),
- TSEncoding.valueOf("RLE"),
- compressionType,
- Collections.emptyMap());
-
- TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
- String[] columns = new String[2];
- columns[0] = "1";
- columns[1] = "2";
-
- InsertRowPlan insertRowPlan =
- new InsertRowPlan(
- new PartialPath("root.laptop.d1.non_aligned_device"),
- time,
- new String[] {"s1", "s2"},
- dataTypes,
- columns,
- false);
- insertRowPlan.setMeasurementMNodes(
- new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
- // call getSeriesSchemasAndReadLockDevice
- IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
- idTable.getSeriesSchemas(insertRowPlan);
-
- IMeasurementMNode s2Node = insertRowPlan.getMeasurementMNodes()[1];
- ILastCacheContainer cacheContainer = s2Node.getLastCacheContainer();
- // last cache
- cacheContainer.updateCachedLast(
- new TimeValuePair(100L, new TsPrimitiveType.TsLong(1L)), false, 0L);
- assertEquals(new TsPrimitiveType.TsLong(1L), cacheContainer.getCachedLast().getValue());
- assertEquals(100L, cacheContainer.getCachedLast().getTimestamp());
-
- cacheContainer.updateCachedLast(
- new TimeValuePair(90L, new TsPrimitiveType.TsLong(2L)), false, 0L);
- assertEquals(new TsPrimitiveType.TsLong(1L), cacheContainer.getCachedLast().getValue());
- assertEquals(100L, cacheContainer.getCachedLast().getTimestamp());
-
- cacheContainer.updateCachedLast(
- new TimeValuePair(110L, new TsPrimitiveType.TsLong(2L)), false, 0L);
- assertEquals(new TsPrimitiveType.TsLong(2L), cacheContainer.getCachedLast().getValue());
- assertEquals(110L, cacheContainer.getCachedLast().getTimestamp());
-
- } catch (MetadataException e) {
- e.printStackTrace();
- fail("throw exception");
- }
- }
-}
+//
+// import org.apache.iotdb.commons.exception.MetadataException;
+// import org.apache.iotdb.commons.path.PartialPath;
+// import org.apache.iotdb.db.conf.IoTDBDescriptor;
+// import org.apache.iotdb.db.engine.trigger.service.TriggerRegistrationService;
+// import org.apache.iotdb.db.exception.StorageEngineException;
+// import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
+// import org.apache.iotdb.db.exception.query.QueryProcessException;
+// import org.apache.iotdb.db.metadata.LocalSchemaProcessor;
+// import org.apache.iotdb.db.metadata.idtable.entry.DeviceIDFactory;
+// import org.apache.iotdb.db.metadata.idtable.entry.DiskSchemaEntry;
+// import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
+// import org.apache.iotdb.db.metadata.idtable.entry.SchemaEntry;
+// import org.apache.iotdb.db.metadata.lastCache.container.ILastCacheContainer;
+// import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+// import org.apache.iotdb.db.qp.Planner;
+// import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+// import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
+// import org.apache.iotdb.db.qp.physical.sys.CreateTriggerPlan;
+// import org.apache.iotdb.db.qp.physical.sys.DropTriggerPlan;
+// import org.apache.iotdb.db.service.IoTDB;
+// import org.apache.iotdb.db.utils.EnvironmentUtils;
+// import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+// import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+// import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+// import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+// import org.apache.iotdb.tsfile.read.TimeValuePair;
+// import org.apache.iotdb.tsfile.utils.Pair;
+// import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
+//
+// import org.junit.After;
+// import org.junit.Before;
+// import org.junit.Test;
+//
+// import java.io.IOException;
+// import java.util.ArrayList;
+// import java.util.Arrays;
+// import java.util.Collection;
+// import java.util.Collections;
+// import java.util.List;
+// import java.util.Set;
+//
+// import static org.junit.Assert.assertEquals;
+// import static org.junit.Assert.assertFalse;
+// import static org.junit.Assert.assertNotEquals;
+// import static org.junit.Assert.assertNotNull;
+// import static org.junit.Assert.assertNull;
+// import static org.junit.Assert.assertTrue;
+// import static org.junit.Assert.fail;
+//
+// public class IDTableTest {
+//
+// private CompressionType compressionType;
+//
+// private boolean isEnableIDTable = false;
+//
+// private String originalDeviceIDTransformationMethod = null;
+//
+// private boolean isEnableIDTableLogFile = false;
+//
+// @Before
+// public void before() {
+// compressionType = TSFileDescriptor.getInstance().getConfig().getCompressor();
+// IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(true);
+// isEnableIDTable = IoTDBDescriptor.getInstance().getConfig().isEnableIDTable();
+// originalDeviceIDTransformationMethod =
+// IoTDBDescriptor.getInstance().getConfig().getDeviceIDTransformationMethod();
+// isEnableIDTableLogFile = IoTDBDescriptor.getInstance().getConfig().isEnableIDTableLogFile();
+//
+// IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(true);
+// IoTDBDescriptor.getInstance().getConfig().setDeviceIDTransformationMethod("SHA256");
+// IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(true);
+// EnvironmentUtils.envSetUp();
+// }
+//
+// @After
+// public void clean() throws IOException, StorageEngineException {
+// IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(isEnableIDTable);
+// IoTDBDescriptor.getInstance()
+// .getConfig()
+// .setDeviceIDTransformationMethod(originalDeviceIDTransformationMethod);
+// IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(isEnableIDTableLogFile);
+// EnvironmentUtils.cleanEnv();
+// }
+//
+// @Test
+// public void testCreateAlignedTimeseriesAndInsert() {
+// LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
+//
+// try {
+// schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
+// CreateAlignedTimeSeriesPlan plan =
+// new CreateAlignedTimeSeriesPlan(
+// new PartialPath("root.laptop.d1.aligned_device"),
+// Arrays.asList("s1", "s2", "s3"),
+// Arrays.asList(
+// TSDataType.valueOf("FLOAT"),
+// TSDataType.valueOf("INT64"),
+// TSDataType.valueOf("INT32")),
+// Arrays.asList(
+// TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"),
+// TSEncoding.valueOf("RLE")),
+// Arrays.asList(compressionType, compressionType, compressionType),
+// null,
+// null,
+// null);
+//
+// schemaProcessor.createAlignedTimeSeries(plan);
+//
+// IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//
+// // construct an insertRowPlan with mismatched data type
+// long time = 1L;
+// TSDataType[] dataTypes =
+// new TSDataType[] {TSDataType.FLOAT, TSDataType.INT64, TSDataType.INT32};
+//
+// String[] columns = new String[3];
+// columns[0] = 2.0 + "";
+// columns[1] = 10000 + "";
+// columns[2] = 100 + "";
+//
+// InsertRowPlan insertRowPlan =
+// new InsertRowPlan(
+// new PartialPath("root.laptop.d1.aligned_device"),
+// time,
+// new String[] {"s1", "s2", "s3"},
+// dataTypes,
+// columns,
+// true);
+// insertRowPlan.setMeasurementMNodes(
+// new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+// idTable.getSeriesSchemas(insertRowPlan);
+//
+// // with type mismatch
+// dataTypes = new TSDataType[] {TSDataType.FLOAT, TSDataType.DOUBLE, TSDataType.INT32};
+// InsertRowPlan insertRowPlan2 =
+// new InsertRowPlan(
+// new PartialPath("root.laptop.d1.aligned_device"),
+// time,
+// new String[] {"s1", "s2", "s3"},
+// dataTypes,
+// columns,
+// true);
+// insertRowPlan2.setMeasurementMNodes(
+// new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+// // we should throw type mismatch exception here
+// try {
+// IoTDBDescriptor.getInstance().getConfig().setEnablePartialInsert(false);
+// idTable.getSeriesSchemas(insertRowPlan2);
+// fail("should throw exception");
+// } catch (DataTypeMismatchException e) {
+// assertEquals(
+// "data type of root.laptop.d1.aligned_device.s2 is not consistent, registered type
+// INT64, inserting type DOUBLE, timestamp 1, value 10000.0",
+// e.getMessage());
+// } catch (Exception e2) {
+// fail("throw wrong exception");
+// }
+//
+// IoTDBDescriptor.getInstance().getConfig().setEnablePartialInsert(true);
+// } catch (Exception e) {
+// e.printStackTrace();
+// fail(e.getMessage());
+// }
+// }
+//
+// @Test
+// public void testCreateAlignedTimeseriesAndInsertNotAlignedData() {
+// LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
+//
+// try {
+// schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
+// CreateAlignedTimeSeriesPlan plan =
+// new CreateAlignedTimeSeriesPlan(
+// new PartialPath("root.laptop.d1.aligned_device"),
+// Arrays.asList("s1", "s2", "s3"),
+// Arrays.asList(
+// TSDataType.valueOf("FLOAT"),
+// TSDataType.valueOf("INT64"),
+// TSDataType.valueOf("INT32")),
+// Arrays.asList(
+// TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"),
+// TSEncoding.valueOf("RLE")),
+// Arrays.asList(compressionType, compressionType, compressionType),
+// null,
+// null,
+// null);
+//
+// schemaProcessor.createAlignedTimeSeries(plan);
+//
+// IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//
+// // construct an insertRowPlan with mismatched data type
+// long time = 1L;
+// TSDataType[] dataTypes =
+// new TSDataType[] {TSDataType.FLOAT, TSDataType.INT64, TSDataType.INT32};
+//
+// String[] columns = new String[3];
+// columns[0] = 2.0 + "";
+// columns[1] = 10000 + "";
+// columns[2] = 100 + "";
+//
+// // non aligned plan
+// InsertRowPlan insertRowPlan =
+// new InsertRowPlan(
+// new PartialPath("root.laptop.d1.aligned_device"),
+// time,
+// new String[] {"s1", "s2", "s3"},
+// dataTypes,
+// columns,
+// false);
+// insertRowPlan.setMeasurementMNodes(
+// new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+// // call getSeriesSchemasAndReadLockDevice
+// try {
+// idTable.getSeriesSchemas(insertRowPlan);
+// fail("should throw exception");
+// } catch (MetadataException e) {
+// assertEquals(
+// "Timeseries under path [root.laptop.d1.aligned_device]'s align value is [true], which
+// is not consistent with insert plan",
+// e.getMessage());
+// } catch (Exception e2) {
+// fail("throw wrong exception");
+// }
+//
+// } catch (Exception e) {
+// e.printStackTrace();
+// fail(e.getMessage());
+// }
+// }
+//
+// @Test
+// public void testCreateTimeseriesAndInsert() {
+// LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
+// try {
+// schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
+// schemaProcessor.createTimeseries(
+// new PartialPath("root.laptop.d1.s0"),
+// TSDataType.valueOf("INT32"),
+// TSEncoding.valueOf("RLE"),
+// compressionType,
+// Collections.emptyMap());
+//
+// IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//
+// long time = 1L;
+// String[] columns = new String[1];
+// columns[0] = 2 + "";
+//
+// // correct insert plan
+// InsertRowPlan insertRowPlan =
+// new InsertRowPlan(
+// new PartialPath("root.laptop.d1"),
+// time,
+// new String[] {"s0"},
+// new TSDataType[] {TSDataType.INT32},
+// columns);
+// insertRowPlan.setMeasurementMNodes(
+// new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+// idTable.getSeriesSchemas(insertRowPlan);
+// assertEquals(insertRowPlan.getMeasurementMNodes()[0].getSchema().getType(),
+// TSDataType.INT32);
+// assertEquals(0, insertRowPlan.getFailedMeasurementNumber());
+//
+// // construct an insertRowPlan with mismatched data type
+// InsertRowPlan insertRowPlan2 =
+// new InsertRowPlan(
+// new PartialPath("root.laptop.d1"),
+// time,
+// new String[] {"s0"},
+// new TSDataType[] {TSDataType.FLOAT},
+// columns);
+// insertRowPlan2.setMeasurementMNodes(
+// new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+// // get series schema
+// idTable.getSeriesSchemas(insertRowPlan2);
+// assertNull(insertRowPlan2.getMeasurementMNodes()[0]);
+// assertEquals(1, insertRowPlan2.getFailedMeasurementNumber());
+//
+// } catch (Exception e) {
+// e.printStackTrace();
+// fail(e.getMessage());
+// }
+// }
+//
+// @Test
+// public void testCreateTimeseriesAndInsertWithAlignedData() {
+// LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
+// try {
+// schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
+// schemaProcessor.createTimeseries(
+// new PartialPath("root.laptop.d1.non_aligned_device.s1"),
+// TSDataType.valueOf("INT32"),
+// TSEncoding.valueOf("RLE"),
+// compressionType,
+// Collections.emptyMap());
+// schemaProcessor.createTimeseries(
+// new PartialPath("root.laptop.d1.non_aligned_device.s2"),
+// TSDataType.valueOf("INT64"),
+// TSEncoding.valueOf("RLE"),
+// compressionType,
+// Collections.emptyMap());
+//
+// // construct an insertRowPlan with mismatched data type
+// long time = 1L;
+// TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
+//
+// String[] columns = new String[2];
+// columns[0] = "1";
+// columns[1] = "2";
+//
+// InsertRowPlan insertRowPlan =
+// new InsertRowPlan(
+// new PartialPath("root.laptop.d1.non_aligned_device"),
+// time,
+// new String[] {"s1", "s2"},
+// dataTypes,
+// columns,
+// true);
+// insertRowPlan.setMeasurementMNodes(
+// new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+// // call getSeriesSchemasAndReadLockDevice
+// IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//
+// try {
+// idTable.getSeriesSchemas(insertRowPlan);
+// fail("should throw exception");
+// } catch (MetadataException e) {
+// assertEquals(
+// "Timeseries under path [root.laptop.d1.non_aligned_device]'s align value is [false],
+// which is not consistent with insert plan",
+// e.getMessage());
+// }
+// } catch (Exception e) {
+// fail("throw wrong exception");
+// }
+// }
+//
+// @Test
+// public void testInsertAndAutoCreate() {
+// LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
+// try {
+// // construct an insertRowPlan with mismatched data type
+// long time = 1L;
+// TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
+//
+// String[] columns = new String[2];
+// columns[0] = "1";
+// columns[1] = "2";
+//
+// InsertRowPlan insertRowPlan =
+// new InsertRowPlan(
+// new PartialPath("root.laptop.d1.non_aligned_device"),
+// time,
+// new String[] {"s1", "s2"},
+// dataTypes,
+// columns,
+// false);
+// insertRowPlan.setMeasurementMNodes(
+// new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+// // call getSeriesSchemasAndReadLockDevice
+// IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//
+// idTable.getSeriesSchemas(insertRowPlan);
+//
+// // check SchemaProcessor
+// IMeasurementMNode s1Node =
+// schemaProcessor.getMeasurementMNode(
+// new PartialPath("root.laptop.d1.non_aligned_device.s1"));
+// assertEquals("s1", s1Node.getName());
+// assertEquals(TSDataType.INT32, s1Node.getSchema().getType());
+// IMeasurementMNode s2Node =
+// schemaProcessor.getMeasurementMNode(
+// new PartialPath("root.laptop.d1.non_aligned_device.s2"));
+// assertEquals("s2", s2Node.getName());
+// assertEquals(TSDataType.INT64, s2Node.getSchema().getType());
+//
+// // insert type mismatch data
+// InsertRowPlan insertRowPlan2 =
+// new InsertRowPlan(
+// new PartialPath("root.laptop.d1.non_aligned_device"),
+// time,
+// new String[] {"s1", "s2"},
+// new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
+// columns,
+// false);
+// insertRowPlan2.setMeasurementMNodes(
+// new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+// idTable.getSeriesSchemas(insertRowPlan2);
+//
+// assertNull(insertRowPlan2.getMeasurementMNodes()[0]);
+// assertEquals(insertRowPlan.getMeasurementMNodes()[1].getSchema().getType(),
+// TSDataType.INT64);
+// assertEquals(1, insertRowPlan2.getFailedMeasurementNumber());
+//
+// // insert aligned data
+// InsertRowPlan insertRowPlan3 =
+// new InsertRowPlan(
+// new PartialPath("root.laptop.d1.non_aligned_device"),
+// time,
+// new String[] {"s1", "s2"},
+// new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
+// columns,
+// true);
+// insertRowPlan3.setMeasurementMNodes(
+// new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+// try {
+// idTable.getSeriesSchemas(insertRowPlan3);
+// fail("should throw exception");
+// } catch (MetadataException e) {
+// assertEquals(
+// "Timeseries under path [root.laptop.d1.non_aligned_device]'s align value is [false],
+// which is not consistent with insert plan",
+// e.getMessage());
+// } catch (Exception e) {
+// fail("throw wrong exception");
+// }
+// } catch (MetadataException e) {
+// e.printStackTrace();
+// fail("throw exception");
+// }
+// }
+//
+// @Test
+// public void testAlignedInsertAndAutoCreate() {
+// LocalSchemaProcessor processor = IoTDB.schemaProcessor;
+// try {
+// // construct an insertRowPlan with mismatched data type
+// long time = 1L;
+// TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
+//
+// String[] columns = new String[2];
+// columns[0] = "1";
+// columns[1] = "2";
+//
+// InsertRowPlan insertRowPlan =
+// new InsertRowPlan(
+// new PartialPath("root.laptop.d1.aligned_device"),
+// time,
+// new String[] {"s1", "s2"},
+// dataTypes,
+// columns,
+// true);
+// insertRowPlan.setMeasurementMNodes(
+// new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+// // call getSeriesSchemasAndReadLockDevice
+// IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//
+// idTable.getSeriesSchemas(insertRowPlan);
+//
+// // check SchemaProcessor
+// IMeasurementMNode s1Node =
+// processor.getMeasurementMNode(new PartialPath("root.laptop.d1.aligned_device.s1"));
+// assertEquals("s1", s1Node.getName());
+// assertEquals(TSDataType.INT32, s1Node.getSchema().getType());
+// IMeasurementMNode s2Node =
+// processor.getMeasurementMNode(new PartialPath("root.laptop.d1.aligned_device.s2"));
+// assertEquals("s2", s2Node.getName());
+// assertEquals(TSDataType.INT64, s2Node.getSchema().getType());
+// assertTrue(s2Node.getParent().isAligned());
+//
+// // insert type mismatch data
+// InsertRowPlan insertRowPlan2 =
+// new InsertRowPlan(
+// new PartialPath("root.laptop.d1.aligned_device"),
+// time,
+// new String[] {"s1", "s2"},
+// new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
+// columns,
+// true);
+// insertRowPlan2.setMeasurementMNodes(
+// new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+// idTable.getSeriesSchemas(insertRowPlan2);
+//
+// assertNull(insertRowPlan2.getMeasurementMNodes()[0]);
+// assertEquals(insertRowPlan.getMeasurementMNodes()[1].getSchema().getType(),
+// TSDataType.INT64);
+// assertEquals(1, insertRowPlan2.getFailedMeasurementNumber());
+//
+// // insert non-aligned data
+// InsertRowPlan insertRowPlan3 =
+// new InsertRowPlan(
+// new PartialPath("root.laptop.d1.aligned_device"),
+// time,
+// new String[] {"s1", "s2"},
+// new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
+// columns,
+// false);
+// insertRowPlan3.setMeasurementMNodes(
+// new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+// try {
+// idTable.getSeriesSchemas(insertRowPlan3);
+// fail("should throw exception");
+// } catch (MetadataException e) {
+// assertEquals(
+// "Timeseries under path [root.laptop.d1.aligned_device]'s align value is [true], which
+// is not consistent with insert plan",
+// e.getMessage());
+// } catch (Exception e) {
+// fail("throw wrong exception");
+// }
+// } catch (MetadataException e) {
+// e.printStackTrace();
+// fail("throw exception");
+// }
+// }
+//
+// @Test
+// public void testTriggerAndInsert() {
+// LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
+// try {
+// long time = 1L;
+//
+// schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
+// schemaProcessor.createTimeseries(
+// new PartialPath("root.laptop.d1.non_aligned_device.s1"),
+// TSDataType.valueOf("INT32"),
+// TSEncoding.valueOf("RLE"),
+// compressionType,
+// Collections.emptyMap());
+// schemaProcessor.createTimeseries(
+// new PartialPath("root.laptop.d1.non_aligned_device.s2"),
+// TSDataType.valueOf("INT64"),
+// TSEncoding.valueOf("RLE"),
+// compressionType,
+// Collections.emptyMap());
+//
+// Planner processor = new Planner();
+//
+// String sql =
+// "CREATE TRIGGER trigger1 BEFORE INSERT ON root.laptop.d1.non_aligned_device.s1 AS
+// 'org.apache.iotdb.db.metadata.idtable.trigger_example.Counter'";
+//
+// CreateTriggerPlan plan = (CreateTriggerPlan) processor.parseSQLToPhysicalPlan(sql);
+//
+// TriggerRegistrationService.getInstance().register(plan);
+//
+// TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
+// String[] columns = new String[2];
+// columns[0] = "1";
+// columns[1] = "2";
+//
+// InsertRowPlan insertRowPlan =
+// new InsertRowPlan(
+// new PartialPath("root.laptop.d1.non_aligned_device"),
+// time,
+// new String[] {"s1", "s2"},
+// dataTypes,
+// columns,
+// false);
+// insertRowPlan.setMeasurementMNodes(
+// new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+// // call getSeriesSchemasAndReadLockDevice
+// IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//
+// idTable.getSeriesSchemas(insertRowPlan);
+//
+// // check SchemaProcessor
+// IMeasurementMNode s1Node =
+// schemaProcessor.getMeasurementMNode(
+// new PartialPath("root.laptop.d1.non_aligned_device.s1"));
+// assertEquals("s1", s1Node.getName());
+// assertEquals(TSDataType.INT32, s1Node.getSchema().getType());
+// assertNotNull(s1Node.getTriggerExecutor());
+//
+// IMeasurementMNode s2Node =
+// schemaProcessor.getMeasurementMNode(
+// new PartialPath("root.laptop.d1.non_aligned_device.s2"));
+// assertEquals("s2", s2Node.getName());
+// assertEquals(TSDataType.INT64, s2Node.getSchema().getType());
+// assertNull(s2Node.getTriggerExecutor());
+//
+// // drop trigger
+// String sql2 = "Drop trigger trigger1";
+//
+// DropTriggerPlan plan2 = (DropTriggerPlan) processor.parseSQLToPhysicalPlan(sql2);
+// TriggerRegistrationService.getInstance().deregister(plan2);
+//
+// idTable.getSeriesSchemas(insertRowPlan);
+// assertNull(s1Node.getTriggerExecutor());
+// } catch (MetadataException | StorageEngineException | QueryProcessException e) {
+// e.printStackTrace();
+// fail("throw exception");
+// }
+// }
+//
+// @Test
+// public void testGetDiskSchemaEntries() {
+// try {
+// IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+// String sgPath = "root.laptop";
+// for (int i = 0; i < 10; i++) {
+// String devicePath = sgPath + ".d" + i;
+// IDeviceID iDeviceID = DeviceIDFactory.getInstance().getDeviceID(devicePath);
+// String measurement = "s" + i;
+// idTable.putSchemaEntry(
+// devicePath,
+// measurement,
+// new SchemaEntry(
+// TSDataType.BOOLEAN,
+// TSEncoding.BITMAP,
+// CompressionType.UNCOMPRESSED,
+// iDeviceID,
+// new PartialPath(devicePath + "." + measurement),
+// false,
+// idTable.getIDiskSchemaManager()),
+// false);
+// SchemaEntry schemaEntry =
+// idTable.getDeviceEntry(iDeviceID.toStringID()).getSchemaEntry(measurement);
+// List<SchemaEntry> schemaEntries = new ArrayList<>();
+// schemaEntries.add(schemaEntry);
+// List<DiskSchemaEntry> diskSchemaEntries = idTable.getDiskSchemaEntries(schemaEntries);
+// assertNotNull(diskSchemaEntries);
+// assertEquals(diskSchemaEntries.size(), 1);
+// assertEquals(diskSchemaEntries.get(0).seriesKey, devicePath + "." + measurement);
+// }
+// } catch (Exception e) {
+// e.printStackTrace();
+// fail("throw exception");
+// }
+// }
+//
+// @Test
+// public void testDeleteTimeseries() {
+// try {
+// IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+// String sgPath = "root.laptop";
+// for (int i = 0; i < 10; i++) {
+// String devicePath = sgPath + ".d" + i;
+// IDeviceID iDeviceID = DeviceIDFactory.getInstance().getDeviceID(devicePath);
+// String measurement = "s" + i;
+// SchemaEntry schemaEntry =
+// new SchemaEntry(
+// TSDataType.BOOLEAN,
+// TSEncoding.BITMAP,
+// CompressionType.UNCOMPRESSED,
+// iDeviceID,
+// new PartialPath(devicePath + "." + measurement),
+// false,
+// idTable.getIDiskSchemaManager());
+// idTable.putSchemaEntry(devicePath, measurement, schemaEntry, false);
+// }
+// List<PartialPath> partialPaths = new ArrayList<>();
+// partialPaths.add(new PartialPath("root.laptop.d0.s0"));
+// partialPaths.add(new PartialPath("root.laptop.d8.s8"));
+// partialPaths.add(new PartialPath("root.laptop.d2.s3"));
+// Pair<Integer, Set<String>> pairs = idTable.deleteTimeseries(partialPaths);
+// assertNotNull(pairs);
+// assertEquals((int) pairs.left, 2);
+// assertTrue(pairs.right.contains("root.laptop.d2.s3"));
+// assertFalse(pairs.right.contains("root.laptop.d0.s0"));
+// assertFalse(pairs.right.contains("root.laptop.d8.s8"));
+// Collection<DiskSchemaEntry> diskSchemaEntries =
+// idTable.getIDiskSchemaManager().getAllSchemaEntry();
+// for (DiskSchemaEntry diskSchemaEntry : diskSchemaEntries) {
+// assertNotEquals("root.laptop.d0.s0", diskSchemaEntry.seriesKey);
+// assertNotEquals("root.laptop.d8.s8", diskSchemaEntry.seriesKey);
+// }
+// assertNull(idTable.getDeviceEntry("root.laptop.d0").getMeasurementMap().get("s0"));
+// assertNull(idTable.getDeviceEntry("root.laptop.d8").getMeasurementMap().get("s1"));
+// } catch (Exception e) {
+// e.printStackTrace();
+// fail("throw exception");
+// }
+// }
+//
+// @Test
+// public void testFlushTimeAndLastCache() {
+// LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
+// try {
+// long time = 1L;
+//
+// schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
+// schemaProcessor.createTimeseries(
+// new PartialPath("root.laptop.d1.non_aligned_device.s1"),
+// TSDataType.valueOf("INT32"),
+// TSEncoding.valueOf("RLE"),
+// compressionType,
+// Collections.emptyMap());
+// schemaProcessor.createTimeseries(
+// new PartialPath("root.laptop.d1.non_aligned_device.s2"),
+// TSDataType.valueOf("INT64"),
+// TSEncoding.valueOf("RLE"),
+// compressionType,
+// Collections.emptyMap());
+//
+// TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
+// String[] columns = new String[2];
+// columns[0] = "1";
+// columns[1] = "2";
+//
+// InsertRowPlan insertRowPlan =
+// new InsertRowPlan(
+// new PartialPath("root.laptop.d1.non_aligned_device"),
+// time,
+// new String[] {"s1", "s2"},
+// dataTypes,
+// columns,
+// false);
+// insertRowPlan.setMeasurementMNodes(
+// new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+// // call getSeriesSchemasAndReadLockDevice
+// IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//
+// idTable.getSeriesSchemas(insertRowPlan);
+//
+// IMeasurementMNode s2Node = insertRowPlan.getMeasurementMNodes()[1];
+// ILastCacheContainer cacheContainer = s2Node.getLastCacheContainer();
+// // last cache
+// cacheContainer.updateCachedLast(
+// new TimeValuePair(100L, new TsPrimitiveType.TsLong(1L)), false, 0L);
+// assertEquals(new TsPrimitiveType.TsLong(1L), cacheContainer.getCachedLast().getValue());
+// assertEquals(100L, cacheContainer.getCachedLast().getTimestamp());
+//
+// cacheContainer.updateCachedLast(
+// new TimeValuePair(90L, new TsPrimitiveType.TsLong(2L)), false, 0L);
+// assertEquals(new TsPrimitiveType.TsLong(1L), cacheContainer.getCachedLast().getValue());
+// assertEquals(100L, cacheContainer.getCachedLast().getTimestamp());
+//
+// cacheContainer.updateCachedLast(
+// new TimeValuePair(110L, new TsPrimitiveType.TsLong(2L)), false, 0L);
+// assertEquals(new TsPrimitiveType.TsLong(2L), cacheContainer.getCachedLast().getValue());
+// assertEquals(110L, cacheContainer.getCachedLast().getTimestamp());
+//
+// } catch (MetadataException e) {
+// e.printStackTrace();
+// fail("throw exception");
+// }
+// }
+// }
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/PlannerTest.java b/server/src/test/java/org/apache/iotdb/db/qp/PlannerTest.java
index 9a621011f8..6590b95a05 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/PlannerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/PlannerTest.java
@@ -25,12 +25,10 @@ import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.exception.StorageEngineException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.exception.sql.SQLParserException;
import org.apache.iotdb.db.metadata.LocalSchemaProcessor;
import org.apache.iotdb.db.qp.executor.PlanExecutor;
import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
import org.apache.iotdb.db.service.IoTDB;
import org.apache.iotdb.db.utils.EnvironmentUtils;
import org.apache.iotdb.service.rpc.thrift.TSLastDataQueryReq;
@@ -54,7 +52,6 @@ import java.util.Collections;
import java.util.List;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
public class PlannerTest {
@@ -166,14 +163,14 @@ public class PlannerTest {
PhysicalPlan plan3 = processor.parseSQLToPhysicalPlan(deleteTSStatement);
assertEquals(OperatorType.DELETE_TIMESERIES, plan3.getOperatorType());
- String insertStatement = "insert into root.vehicle.d0(timestamp,s0) values(10,100)";
- PhysicalPlan plan4 = processor.parseSQLToPhysicalPlan(insertStatement);
- assertEquals(OperatorType.INSERT, plan4.getOperatorType());
-
- String deleteStatement =
- "DELETE FROM root.device0.sensor0,root.device0.sensor1 WHERE time <= 5000";
- PhysicalPlan plan6 = processor.parseSQLToPhysicalPlan(deleteStatement);
- assertEquals(OperatorType.DELETE, plan6.getOperatorType());
+ // String insertStatement = "insert into root.vehicle.d0(timestamp,s0) values(10,100)";
+ // PhysicalPlan plan4 = processor.parseSQLToPhysicalPlan(insertStatement);
+ // assertEquals(OperatorType.INSERT, plan4.getOperatorType());
+ //
+ // String deleteStatement =
+ // "DELETE FROM root.device0.sensor0,root.device0.sensor1 WHERE time <= 5000";
+ // PhysicalPlan plan6 = processor.parseSQLToPhysicalPlan(deleteStatement);
+ // assertEquals(OperatorType.DELETE, plan6.getOperatorType());
String queryStatement1 =
"select * from root.vehicle.** where root.vehicle.device1.sensor1 > 50";
@@ -195,9 +192,9 @@ public class PlannerTest {
PhysicalPlan plan10 = processor.parseSQLToPhysicalPlan(fillStatement);
assertEquals(OperatorType.FILL, plan10.getOperatorType());
- String insertTimeStatement = "insert into root.vehicle.d0(time,s0) values(10,100)";
- PhysicalPlan plan11 = processor.parseSQLToPhysicalPlan(insertTimeStatement);
- assertEquals(OperatorType.INSERT, plan11.getOperatorType());
+ // String insertTimeStatement = "insert into root.vehicle.d0(time,s0) values(10,100)";
+ // PhysicalPlan plan11 = processor.parseSQLToPhysicalPlan(insertTimeStatement);
+ // assertEquals(OperatorType.INSERT, plan11.getOperatorType());
String createTSStatement2 =
"create timeseries root.a.b.d_1.`1s` with datatype=FLOAT,encoding=RLE";
@@ -209,16 +206,18 @@ public class PlannerTest {
PhysicalPlan plan13 = processor.parseSQLToPhysicalPlan(queryStatement2);
assertEquals(OperatorType.QUERY, plan13.getOperatorType());
- String insertStatementException = "insert into root.vehicle.d0(timestamp,s0,s1) values(10,100)";
- try {
- processor.parseSQLToPhysicalPlan(insertStatementException);
- } catch (Exception e) {
- assertEquals(
- new SQLParserException(
- "the measurementList's size 2 is not consistent with the valueList's size 1")
- .getMessage(),
- e.getMessage());
- }
+ // String insertStatementException = "insert into root.vehicle.d0(timestamp,s0,s1)
+ // values(10,100)";
+ // try {
+ // processor.parseSQLToPhysicalPlan(insertStatementException);
+ // } catch (Exception e) {
+ // assertEquals(
+ // new SQLParserException(
+ // "the measurementList's size 2 is not consistent with the valueList's size
+ // 1")
+ // .getMessage(),
+ // e.getMessage());
+ // }
}
@Test
@@ -235,28 +234,6 @@ public class PlannerTest {
processor.parseSQLToPhysicalPlan(createTSStatement);
}
- @Test
- public void insertStatementWithNullValue() throws QueryProcessException {
- String createTSStatement = "insert into root.vehicle.d0(time,s0) values(10,NaN)";
- PhysicalPlan physicalPlan = processor.parseSQLToPhysicalPlan(createTSStatement);
-
- assertTrue(physicalPlan instanceof InsertRowPlan);
- assertEquals("NaN", ((InsertRowPlan) physicalPlan).getValues()[0]);
- // Later we will use Double.parseDouble so we have to ensure that it is parsed right
- assertEquals(Double.NaN, Double.parseDouble("NaN"), 1e-15);
- }
-
- @Test
- public void insertStatementWithNegativeTimeStamp()
- throws QueryProcessException, MetadataException {
- String createTSStatement = "insert into root.vehicle.d0(time,s0) values(-1000, 111)";
- PhysicalPlan physicalPlan = processor.parseSQLToPhysicalPlan(createTSStatement);
-
- assertTrue(physicalPlan instanceof InsertRowPlan);
- assertEquals(-1000, ((InsertRowPlan) physicalPlan).getTime());
- assertEquals("111", ((InsertRowPlan) physicalPlan).getValues()[0]);
- }
-
@Test
public void rawDataQueryReqToPhysicalPlanTest()
throws QueryProcessException, IllegalPathException {
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/writelog/LogWriterReaderTest.java b/server/src/test/java/org/apache/iotdb/db/utils/writelog/LogWriterReaderTest.java
index fa2969c3ff..71e48b6183 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/writelog/LogWriterReaderTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/writelog/LogWriterReaderTest.java
@@ -22,9 +22,10 @@ package org.apache.iotdb.db.utils.writelog;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import org.junit.Before;
import org.junit.Test;
@@ -51,24 +52,33 @@ public class LogWriterReaderTest {
if (new File(filePath).exists()) {
new File(filePath).delete();
}
- InsertRowPlan insertRowPlan1 =
- new InsertRowPlan(
- new PartialPath("d1"),
- 10L,
- new String[] {"s1", "s2"},
- new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
- new String[] {"1", "2"});
- InsertRowPlan insertRowPlan2 =
- new InsertRowPlan(
- new PartialPath("d1"),
- 10L,
- new String[] {"s1", "s2"},
- new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
- new String[] {"1", "2"});
- DeletePlan deletePlan = new DeletePlan(Long.MIN_VALUE, 10L, new PartialPath("root.d1.s1"));
- plans.add(insertRowPlan1);
- plans.add(insertRowPlan2);
- plans.add(deletePlan);
+ CreateTimeSeriesPlan plan1 =
+ new CreateTimeSeriesPlan(
+ new PartialPath("d1.s1"),
+ TSDataType.INT64,
+ TSEncoding.PLAIN,
+ CompressionType.SNAPPY,
+ null,
+ null,
+ null,
+ null);
+
+ CreateTimeSeriesPlan plan2 =
+ new CreateTimeSeriesPlan(
+ new PartialPath("d1.s2"),
+ TSDataType.INT64,
+ TSEncoding.PLAIN,
+ CompressionType.SNAPPY,
+ null,
+ null,
+ null,
+ null);
+
+ List<PartialPath> paths = new ArrayList<>();
+ paths.add(new PartialPath("d1.s1"));
+ paths.add(new PartialPath("d1.s2"));
+ plans.add(plan1);
+ plans.add(plan2);
for (PhysicalPlan plan : plans) {
plan.serialize(logsBuffer);
}
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/writelog/MultiFileLogReaderTest.java b/server/src/test/java/org/apache/iotdb/db/utils/writelog/MultiFileLogReaderTest.java
index 5e7af41382..14b0d6f377 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/writelog/MultiFileLogReaderTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/writelog/MultiFileLogReaderTest.java
@@ -22,7 +22,10 @@ package org.apache.iotdb.db.utils.writelog;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import org.apache.commons.io.FileUtils;
import org.junit.After;
@@ -50,7 +53,15 @@ public class MultiFileLogReaderTest {
logFiles[i] = new File(i + ".log");
for (int j = 0; j < logsPerFile; j++) {
fileLogs[i][j] =
- new DeletePlan(Long.MIN_VALUE, i * logsPerFile + j, new PartialPath("path" + j));
+ new CreateTimeSeriesPlan(
+ new PartialPath("root.sg.d1.s" + j),
+ TSDataType.INT32,
+ TSEncoding.PLAIN,
+ CompressionType.SNAPPY,
+ null,
+ null,
+ null,
+ null);
}
ByteBuffer buffer = ByteBuffer.allocate(64 * 1024);
diff --git a/server/src/test/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoerTest.java b/server/src/test/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoerTest.java
index b0197a7a3b..cfbd8e095e 100644
--- a/server/src/test/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoerTest.java
@@ -616,7 +616,7 @@ public class TsFilePlanRedoerTest {
tsFileResource.updateStartTime(DEVICE2_NAME, 3);
tsFileResource.updateEndTime(DEVICE2_NAME, 4);
- // generate DeletePlan
+ // generate DeleteDataNode
DeleteDataNode deleteDataNode =
new DeleteDataNode(
new PlanNodeId(""),
@@ -624,7 +624,7 @@ public class TsFilePlanRedoerTest {
Long.MIN_VALUE,
Long.MAX_VALUE);
- // redo DeletePlan, vsg processor is used to test IdTable, don't test IdTable here
+ // redo DeleteDataNode, vsg processor is used to test IdTable, don't test IdTable here
File modsFile = new File(FILE_NAME.concat(ModificationFile.FILE_SUFFIX));
assertFalse(modsFile.exists());
TsFilePlanRedoer planRedoer = new TsFilePlanRedoer(tsFileResource, false, null);