[IOTDB-1199] Support aligned timeseries and device template (#2802)
Co-authored-by: 151250176 <151250176@smail.nju.edu.cn>
Co-authored-by: samperson1997 <szs19@mails.tsinghua.edu.cn>
Co-authored-by: Jackie Tien <JackieTien@foxmail.com>
Co-authored-by: LebronAl <TXYPotato@gmail.com>
Co-authored-by: Alima777 <wxw19981014@gmail.com>
Co-authored-by: Steve Yurong Su <steveyurongsu@outlook.com>
Co-authored-by: chaow <xunmengzhuiyi@gmail.com>
Co-authored-by: Jialin Qiao <qjl16@mails.tsinghua.edu.cn>
diff --git a/.github/workflows/main-unix.yml b/.github/workflows/main-unix.yml
index 15aa14e..c1183e1 100644
--- a/.github/workflows/main-unix.yml
+++ b/.github/workflows/main-unix.yml
@@ -8,6 +8,7 @@
branches:
- master
- 'rel/*'
+ - Vector
paths-ignore:
- 'docs/**'
pull_request:
@@ -15,6 +16,7 @@
- master
- 'rel/*'
- cluster_new
+ - Vector
paths-ignore:
- 'docs/**'
# allow manually run the action:
diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
index 6908655..a7b2fc9 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
@@ -374,12 +374,21 @@
;
insertColumnsSpec
- : LR_BRACKET (TIMESTAMP|TIME) (COMMA nodeNameWithoutStar)+ RR_BRACKET
+ : LR_BRACKET (TIMESTAMP|TIME) (COMMA measurementName)+ RR_BRACKET
+ ;
+measurementName
+ : nodeNameWithoutStar
+ | LR_BRACKET nodeNameWithoutStar (COMMA nodeNameWithoutStar)+ RR_BRACKET
;
insertValuesSpec
- : LR_BRACKET dateFormat (COMMA constant)+ RR_BRACKET
- | LR_BRACKET INT (COMMA constant)+ RR_BRACKET
+ : LR_BRACKET dateFormat (COMMA measurementValue)+ RR_BRACKET
+ | LR_BRACKET INT (COMMA measurementValue)+ RR_BRACKET
+ ;
+
+measurementValue
+ : constant
+ | LR_BRACKET constant (COMMA constant)+ RR_BRACKET
;
setCol
@@ -664,6 +673,7 @@
| MINUS? INT
| stringLiteral
| booleanClause
+ | NULL
;
booleanClause
@@ -1290,6 +1300,10 @@
: E X P L A I N
;
+NULL
+ : N U L L
+ ;
+
//============================
// End of the keywords list
//============================
diff --git a/cluster/src/assembly/resources/conf/iotdb-cluster.properties b/cluster/src/assembly/resources/conf/iotdb-cluster.properties
index 079b124..62014ca 100644
--- a/cluster/src/assembly/resources/conf/iotdb-cluster.properties
+++ b/cluster/src/assembly/resources/conf/iotdb-cluster.properties
@@ -65,7 +65,7 @@
# max_concurrent_client_num=10000
# number of replications for one partition
-default_replica_num=3
+default_replica_num=1
# cluster name to identify different clusters
# all node's cluster_name in one cluster are the same
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/ClientMain.java b/cluster/src/main/java/org/apache/iotdb/cluster/ClientMain.java
index 7543f4a..cf30381 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/ClientMain.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/ClientMain.java
@@ -41,6 +41,7 @@
import org.apache.iotdb.service.rpc.thrift.TSStatus;
import org.apache.iotdb.session.SessionDataSet;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.commons.cli.CommandLine;
@@ -115,7 +116,7 @@
private static final TSDataType[] DATA_TYPES = new TSDataType[] {TSDataType.DOUBLE};
- private static List<MeasurementSchema> schemas;
+ private static List<IMeasurementSchema> schemas;
private static final String[] DATA_QUERIES =
new String[] {
@@ -364,7 +365,7 @@
private static void registerTimeseries(long sessionId, Client client) throws TException {
TSCreateTimeseriesReq req = new TSCreateTimeseriesReq();
req.setSessionId(sessionId);
- for (MeasurementSchema schema : schemas) {
+ for (IMeasurementSchema schema : schemas) {
req.setDataType(schema.getType().ordinal());
req.setEncoding(schema.getEncodingType().ordinal());
req.setCompressor(schema.getCompressor().ordinal());
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/ClusterMain.java b/cluster/src/main/java/org/apache/iotdb/cluster/ClusterMain.java
index 27aa61b..f39e9a6 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/ClusterMain.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/ClusterMain.java
@@ -103,7 +103,7 @@
try {
metaServer = new MetaClusterServer();
startServerCheck();
- // preStartCustomize();
+ preStartCustomize();
metaServer.start();
metaServer.buildCluster();
} catch (TTransportException
@@ -117,7 +117,7 @@
} else if (MODE_ADD.equals(mode)) {
try {
metaServer = new MetaClusterServer();
- // preStartCustomize();
+ preStartCustomize();
metaServer.start();
metaServer.joinCluster();
} catch (TTransportException
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
index 6d73265..a1145ba 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptor.java
@@ -62,7 +62,7 @@
import org.apache.iotdb.tsfile.read.filter.basic.Filter;
import org.apache.iotdb.tsfile.read.filter.factory.FilterFactory;
import org.apache.iotdb.tsfile.read.filter.operator.AndFilter;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
import org.apache.thrift.TException;
@@ -285,10 +285,10 @@
return response.get();
}
- public static List<MeasurementSchema> pullMeasurementSchema(
+ public static List<IMeasurementSchema> pullMeasurementSchema(
AsyncDataClient client, PullSchemaRequest pullSchemaRequest)
throws TException, InterruptedException {
- AtomicReference<List<MeasurementSchema>> measurementSchemas = new AtomicReference<>();
+ AtomicReference<List<IMeasurementSchema>> measurementSchemas = new AtomicReference<>();
client.pullMeasurementSchema(
pullSchemaRequest,
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java
index 241dc82..43e2ee5 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java
@@ -41,7 +41,7 @@
@ClusterConsistent private boolean isRpcThriftCompressionEnabled = false;
private int maxConcurrentClientNum = 10000;
- @ClusterConsistent private int replicationNum = 3;
+ @ClusterConsistent private int replicationNum = 1;
@ClusterConsistent private String clusterName = "default";
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java b/cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
index 3c305a8..3cc64ba 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/coordinator/Coordinator.java
@@ -40,12 +40,14 @@
import org.apache.iotdb.db.exception.metadata.PathNotExistException;
import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.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.InsertMultiTabletPlan;
import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
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.SetDeviceTemplatePlan;
import org.apache.iotdb.db.qp.physical.sys.CreateMultiTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
@@ -181,12 +183,31 @@
metaGroupMember.waitLeader();
return metaGroupMember.forwardPlan(plan, metaGroupMember.getLeader(), null);
}
-
+ try {
+ createSchemaIfNecessary(plan);
+ } catch (MetadataException | CheckConsistencyException e) {
+ logger.error("{}: Cannot find storage groups for {}", name, plan);
+ return StatusUtils.NO_STORAGE_GROUP;
+ }
List<PartitionGroup> globalGroups = metaGroupMember.getPartitionTable().getGlobalGroups();
logger.debug("Forwarding global data plan {} to {} groups", plan, globalGroups.size());
return forwardPlan(globalGroups, plan);
}
+ public void createSchemaIfNecessary(PhysicalPlan plan)
+ throws MetadataException, CheckConsistencyException {
+ if (plan instanceof SetDeviceTemplatePlan) {
+ try {
+ IoTDB.metaManager.getStorageGroupPath(
+ new PartialPath(((SetDeviceTemplatePlan) plan).getPrefixPath()));
+ } catch (IllegalPathException e) {
+ // the plan has been checked
+ } catch (StorageGroupNotSetException e) {
+ ((CMManager) IoTDB.metaManager).createSchema(plan);
+ }
+ }
+ }
+
/**
* A partitioned plan (like batch insertion) will be split into several sub-plans, each belongs to
* a data group. And these sub-plans will be sent to and executed on the corresponding groups
@@ -244,19 +265,29 @@
for (PartitionGroup partitionGroup : partitionGroups) {
if (partitionGroup.contains(thisNode)) {
// the query should be handled by a group the local node is in, handle it with in the group
- logger.debug("Execute {} in a local group of {}", plan, partitionGroup.getHeader());
status =
metaGroupMember
.getLocalDataMember(partitionGroup.getHeader())
.executeNonQueryPlan(plan);
+ logger.debug(
+ "Execute {} in a local group of {} with status {}",
+ plan,
+ partitionGroup.getHeader(),
+ status);
} else {
// forward the query to the group that should handle it
- logger.debug("Forward {} to a remote group of {}", plan, partitionGroup.getHeader());
status = forwardPlan(plan, partitionGroup);
+ logger.debug(
+ "Forward {} to a remote group of {} with status {}",
+ plan,
+ partitionGroup.getHeader(),
+ status);
}
if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()
- && (!(plan instanceof DeleteTimeSeriesPlan)
- || status.getCode() != TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode())) {
+ && !(plan instanceof SetDeviceTemplatePlan
+ && status.getCode() == TSStatusCode.DUPLICATED_TEMPLATE.getStatusCode())
+ && !(plan instanceof DeleteTimeSeriesPlan
+ && status.getCode() == TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode())) {
// execution failed, record the error message
errorCodePartitionGroups.add(
String.format(
@@ -324,13 +355,19 @@
status = forwardToMultipleGroup(planGroupMap);
}
}
- if (plan instanceof InsertPlan
- && status.getCode() == TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode()
- && ClusterDescriptor.getInstance().getConfig().isEnableAutoCreateSchema()) {
- TSStatus tmpStatus = createTimeseriesForFailedInsertion(planGroupMap, ((InsertPlan) plan));
- if (tmpStatus != null) {
- status = tmpStatus;
+ boolean hasCreated = false;
+ try {
+ if (plan instanceof InsertPlan
+ && status.getCode() == TSStatusCode.TIMESERIES_NOT_EXIST.getStatusCode()
+ && ClusterDescriptor.getInstance().getConfig().isEnableAutoCreateSchema()) {
+ hasCreated = createTimeseriesForFailedInsertion(((InsertPlan) plan));
}
+ } catch (MetadataException | CheckConsistencyException e) {
+ logger.error("{}: Cannot auto-create timeseries for {}", name, plan, e);
+ return StatusUtils.getStatus(StatusUtils.EXECUTE_STATEMENT_ERROR, e.getMessage());
+ }
+ if (hasCreated) {
+ status = forwardPlan(planGroupMap, plan);
}
if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()
&& status.isSetRedirectNode()) {
@@ -340,24 +377,13 @@
return status;
}
- private TSStatus createTimeseriesForFailedInsertion(
- Map<PhysicalPlan, PartitionGroup> planGroupMap, InsertPlan plan) {
+ private boolean createTimeseriesForFailedInsertion(InsertPlan plan)
+ throws CheckConsistencyException, IllegalPathException {
// try to create timeseries
if (plan.getFailedMeasurements() != null) {
plan.getPlanFromFailed();
}
- boolean hasCreate;
- try {
- hasCreate = ((CMManager) IoTDB.metaManager).createTimeseries(plan);
- } catch (IllegalPathException | CheckConsistencyException e) {
- return StatusUtils.getStatus(StatusUtils.EXECUTE_STATEMENT_ERROR, e.getMessage());
- }
- if (hasCreate) {
- return forwardPlan(planGroupMap, plan);
- } else {
- logger.error("{}, Cannot auto create timeseries.", thisNode);
- }
- return null;
+ return ((CMManager) IoTDB.metaManager).createTimeseries(plan);
}
private TSStatus forwardToSingleGroup(Map.Entry<PhysicalPlan, PartitionGroup> entry) {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/BaseApplier.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/BaseApplier.java
index 460ba6e..6333717 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/BaseApplier.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/applier/BaseApplier.java
@@ -31,6 +31,7 @@
import org.apache.iotdb.db.exception.metadata.MetadataException;
import org.apache.iotdb.db.exception.metadata.PathNotExistException;
import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.metadata.PartialPath;
import org.apache.iotdb.db.qp.executor.PlanExecutor;
@@ -72,7 +73,8 @@
try {
getQueryExecutor().processNonQuery(plan);
} catch (QueryProcessException e) {
- if (e.getCause() instanceof StorageGroupNotSetException) {
+ if (e.getCause() instanceof StorageGroupNotSetException
+ || e.getCause() instanceof UndefinedTemplateException) {
executeAfterSync(plan);
} else if (e instanceof BatchProcessException) {
logger.warn("Exception occurred while processing non-query. ", e);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
index 6e706cc..4fc6bbf 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
@@ -45,8 +45,10 @@
import org.apache.iotdb.db.metadata.MManager;
import org.apache.iotdb.db.metadata.MetaUtils;
import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.VectorPartialPath;
import org.apache.iotdb.db.metadata.mnode.MNode;
import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.template.Template;
import org.apache.iotdb.db.qp.constant.SQLConstant;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
import org.apache.iotdb.db.qp.physical.crud.InsertMultiTabletPlan;
@@ -55,6 +57,8 @@
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.SetDeviceTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateMultiTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
@@ -76,8 +80,9 @@
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.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
import org.apache.thrift.TException;
import org.slf4j.Logger;
@@ -94,6 +99,7 @@
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
+import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
@@ -185,12 +191,25 @@
@Override
public TSDataType getSeriesType(PartialPath path) throws MetadataException {
+
+ if (path.equals(SQLConstant.TIME_PATH)) {
+ return TSDataType.INT64;
+ }
+
+ if (path instanceof VectorPartialPath) {
+ if (((VectorPartialPath) path).getSubSensorsPathList().size() != 1) {
+ return TSDataType.VECTOR;
+ } else {
+ path = ((VectorPartialPath) path).getSubSensorsPathList().get(0);
+ }
+ }
+
// try remote cache first
try {
cacheLock.readLock().lock();
MeasurementMNode measurementMNode = mRemoteMetaCache.get(path);
if (measurementMNode != null) {
- return measurementMNode.getSchema().getType();
+ return measurementMNode.getDataType(path.getMeasurement());
}
} finally {
cacheLock.readLock().unlock();
@@ -202,15 +221,24 @@
seriesType = super.getSeriesType(path);
} catch (PathNotExistException e) {
// pull from remote node
- List<MeasurementSchema> schemas =
+ List<IMeasurementSchema> schemas =
metaPuller.pullMeasurementSchemas(Collections.singletonList(path));
if (!schemas.isEmpty()) {
- MeasurementSchema measurementSchema = schemas.get(0);
+ IMeasurementSchema measurementSchema = schemas.get(0);
MeasurementMNode measurementMNode =
new MeasurementMNode(
null, measurementSchema.getMeasurementId(), measurementSchema, null);
- cacheMeta(path, measurementMNode);
- return schemas.get(0).getType();
+ if (measurementSchema instanceof VectorMeasurementSchema) {
+ for (String subSensorId : measurementSchema.getValueMeasurementIdList()) {
+ cacheMeta(new PartialPath(path.getDevice(), subSensorId), measurementMNode);
+ }
+ cacheMeta(
+ new PartialPath(path.getDevice(), measurementSchema.getMeasurementId()),
+ measurementMNode);
+ } else {
+ cacheMeta(path, measurementMNode);
+ }
+ return measurementMNode.getDataType(path.getMeasurement());
} else {
throw e;
}
@@ -218,6 +246,60 @@
return seriesType;
}
+ @Override
+ public Pair<List<PartialPath>, Map<String, Integer>> getSeriesSchemas(List<PartialPath> fullPaths)
+ throws MetadataException {
+ Map<MNode, PartialPath> nodeToPartialPath = new LinkedHashMap<>();
+ Map<MNode, List<Integer>> nodeToIndex = new LinkedHashMap<>();
+ for (int i = 0; i < fullPaths.size(); i++) {
+ PartialPath path = fullPaths.get(i);
+ MeasurementMNode node = getMeasurementMNode(path);
+ super.getNodeToPartialPath(node, nodeToPartialPath, nodeToIndex, path, i);
+ }
+ return getPair(fullPaths, nodeToPartialPath, nodeToIndex);
+ }
+
+ @Override
+ public IMeasurementSchema getSeriesSchema(PartialPath fullPath) throws MetadataException {
+ return super.getSeriesSchema(fullPath, getMeasurementMNode(fullPath));
+ }
+
+ private MeasurementMNode getMeasurementMNode(PartialPath fullPath) throws MetadataException {
+ MeasurementMNode node = null;
+ // try remote cache first
+ try {
+ cacheLock.readLock().lock();
+ MeasurementMNode measurementMNode = mRemoteMetaCache.get(fullPath);
+ if (measurementMNode != null) {
+ node = measurementMNode;
+ }
+ } finally {
+ cacheLock.readLock().unlock();
+ }
+
+ if (node == null) {
+ // try local MTree
+ try {
+ node = (MeasurementMNode) super.getNodeByPath(fullPath);
+ } catch (PathNotExistException e) {
+ // pull from remote node
+ List<IMeasurementSchema> schemas =
+ metaPuller.pullMeasurementSchemas(Collections.singletonList(fullPath));
+ if (!schemas.isEmpty()) {
+ IMeasurementSchema measurementSchema = schemas.get(0);
+ MeasurementMNode measurementMNode =
+ new MeasurementMNode(
+ null, measurementSchema.getMeasurementId(), measurementSchema, null);
+ cacheMeta(fullPath, measurementMNode);
+ node = measurementMNode;
+ } else {
+ throw e;
+ }
+ }
+ }
+ return node;
+ }
+
/**
* the {@link org.apache.iotdb.db.writelog.recover.LogReplayer#replayLogs(Supplier)} will call
* this to get schema after restart we should retry to get schema util we get the schema.
@@ -282,8 +364,8 @@
for (String s : measurementList) {
schemasToPull.add(deviceId.concatNode(s));
}
- List<MeasurementSchema> schemas = metaPuller.pullMeasurementSchemas(schemasToPull);
- for (MeasurementSchema schema : schemas) {
+ List<IMeasurementSchema> schemas = metaPuller.pullMeasurementSchemas(schemasToPull);
+ for (IMeasurementSchema schema : schemas) {
// TODO-Cluster: also pull alias?
// take care, the pulled schema's measurement Id is only series name
MeasurementMNode measurementMNode =
@@ -331,7 +413,8 @@
}
@Override
- public MNode getSeriesSchemasAndReadLockDevice(InsertPlan plan) throws MetadataException {
+ public MNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
+ throws MetadataException, IOException {
MeasurementMNode[] measurementMNodes = new MeasurementMNode[plan.getMeasurements().length];
int nonExistSchemaIndex =
getMNodesLocally(plan.getDeviceId(), plan.getMeasurements(), measurementMNodes);
@@ -345,10 +428,10 @@
}
@Override
- public MeasurementSchema getSeriesSchema(PartialPath device, String measurement)
+ public IMeasurementSchema getSeriesSchema(PartialPath device, String measurement)
throws MetadataException {
try {
- MeasurementSchema measurementSchema = super.getSeriesSchema(device, measurement);
+ IMeasurementSchema measurementSchema = super.getSeriesSchema(device, measurement);
if (measurementSchema != null) {
return measurementSchema;
}
@@ -404,6 +487,16 @@
}
}
+ @Override
+ public Pair<MNode, Template> getDeviceNodeWithAutoCreate(PartialPath path)
+ throws MetadataException, IOException {
+ return getDeviceNodeWithAutoCreate(
+ path,
+ ClusterDescriptor.getInstance().getConfig().isEnableAutoCreateSchema(),
+ false,
+ config.getDefaultStorageGroupLevel());
+ }
+
private static class RemoteMetaCache extends LRUCache<PartialPath, MeasurementMNode> {
RemoteMetaCache(int cacheSize) {
@@ -468,6 +561,11 @@
} else if (plan instanceof CreateTimeSeriesPlan) {
storageGroups.addAll(
getStorageGroups(Collections.singletonList(((CreateTimeSeriesPlan) plan).getPath())));
+ } else if (plan instanceof SetDeviceTemplatePlan) {
+ storageGroups.addAll(
+ getStorageGroups(
+ Collections.singletonList(
+ new PartialPath(((SetDeviceTemplatePlan) plan).getPrefixPath()))));
} else {
storageGroups.addAll(getStorageGroups(plan.getPaths()));
}
@@ -613,9 +711,16 @@
logger.error("Failed to infer storage group from deviceId {}", deviceId);
return false;
}
+ boolean hasVector = false;
for (String measurementId : insertPlan.getMeasurements()) {
+ if (measurementId.contains("(") && measurementId.contains(",")) {
+ hasVector = true;
+ }
seriesList.add(deviceId.getFullPath() + TsFileConstant.PATH_SEPARATOR + measurementId);
}
+ if (hasVector) {
+ return createAlignedTimeseries(seriesList, (InsertTabletPlan) insertPlan);
+ }
PartitionGroup partitionGroup =
metaGroupMember.getPartitionTable().route(storageGroupName.getFullPath(), 0);
List<String> unregisteredSeriesList = getUnregisteredSeriesList(seriesList, partitionGroup);
@@ -627,6 +732,51 @@
return createTimeseries(unregisteredSeriesList, seriesList, insertPlan);
}
+ private boolean createAlignedTimeseries(List<String> seriesList, InsertTabletPlan insertPlan)
+ throws IllegalPathException {
+ List<String> measurements = new ArrayList<>();
+ for (String series : seriesList) {
+ measurements.addAll(MetaUtils.getMeasurementsInPartialPath(new PartialPath(series)));
+ }
+
+ List<TSDataType> dataTypes = new ArrayList<>();
+ List<TSEncoding> encodings = new ArrayList<>();
+ for (TSDataType dataType : insertPlan.getDataTypes()) {
+ dataTypes.add(dataType);
+ encodings.add(getDefaultEncoding(dataType));
+ }
+
+ CreateAlignedTimeSeriesPlan plan =
+ new CreateAlignedTimeSeriesPlan(
+ insertPlan.getDeviceId(),
+ measurements,
+ dataTypes,
+ encodings,
+ TSFileDescriptor.getInstance().getConfig().getCompressor(),
+ null);
+ TSStatus result;
+ try {
+ result = coordinator.processPartitionedPlan(plan);
+ } catch (UnsupportedPlanException e) {
+ logger.error(
+ "Failed to create timeseries {} automatically. Unsupported plan exception {} ",
+ plan,
+ e.getMessage());
+ return false;
+ }
+ if (result.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()
+ && result.getCode() != TSStatusCode.PATH_ALREADY_EXIST_ERROR.getStatusCode()
+ && result.getCode() != TSStatusCode.NEED_REDIRECTION.getStatusCode()) {
+ logger.error(
+ "{} failed to execute create timeseries {}: {}",
+ metaGroupMember.getThisNode(),
+ plan,
+ result);
+ return false;
+ }
+ return true;
+ }
+
/**
* create timeseries from paths in "unregisteredSeriesList". If data types are provided by the
* InsertPlan, use them, otherwise infer the types from the values. Use default encodings and
@@ -664,6 +814,7 @@
plan.setDataTypes(dataTypes);
plan.setEncodings(encodings);
plan.setCompressors(compressionTypes);
+
TSStatus result;
try {
result = coordinator.processPartitionedPlan(plan);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/MetaPuller.java b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/MetaPuller.java
index e524772..35a74b8 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/MetaPuller.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/MetaPuller.java
@@ -34,7 +34,9 @@
import org.apache.iotdb.db.exception.metadata.MetadataException;
import org.apache.iotdb.db.metadata.PartialPath;
import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
import org.apache.thrift.TException;
import org.slf4j.Logger;
@@ -77,7 +79,7 @@
* Pull the all timeseries schemas of given prefixPaths from remote nodes. All prefixPaths must
* contain the storage group.
*/
- List<MeasurementSchema> pullMeasurementSchemas(List<PartialPath> prefixPaths)
+ List<IMeasurementSchema> pullMeasurementSchemas(List<PartialPath> prefixPaths)
throws MetadataException {
logger.debug("{}: Pulling timeseries schemas of {}", metaGroupMember.getName(), prefixPaths);
// split the paths by the data groups that will hold them
@@ -88,7 +90,7 @@
partitionGroupPathMap.computeIfAbsent(partitionGroup, g -> new ArrayList<>()).add(prefixPath);
}
- List<MeasurementSchema> schemas = new ArrayList<>();
+ List<IMeasurementSchema> schemas = new ArrayList<>();
// pull timeseries schema from every group involved
if (logger.isDebugEnabled()) {
logger.debug(
@@ -127,7 +129,7 @@
private void pullMeasurementSchemas(
PartitionGroup partitionGroup,
List<PartialPath> prefixPaths,
- List<MeasurementSchema> results) {
+ List<IMeasurementSchema> results) {
if (partitionGroup.contains(metaGroupMember.getThisNode())) {
// the node is in the target group, synchronize with leader should be enough
try {
@@ -166,7 +168,7 @@
}
private boolean pullMeasurementSchemas(
- Node node, PullSchemaRequest request, List<MeasurementSchema> results) {
+ Node node, PullSchemaRequest request, List<IMeasurementSchema> results) {
if (logger.isDebugEnabled()) {
logger.debug(
"{}: Pulling timeseries schemas of {} and other {} paths from {}",
@@ -176,7 +178,7 @@
node);
}
- List<MeasurementSchema> schemas = null;
+ List<IMeasurementSchema> schemas = null;
try {
schemas = pullMeasurementSchemas(node, request);
} catch (IOException | TException e) {
@@ -215,9 +217,9 @@
return false;
}
- private List<MeasurementSchema> pullMeasurementSchemas(Node node, PullSchemaRequest request)
+ private List<IMeasurementSchema> pullMeasurementSchemas(Node node, PullSchemaRequest request)
throws TException, InterruptedException, IOException {
- List<MeasurementSchema> schemas;
+ List<IMeasurementSchema> schemas;
if (ClusterDescriptor.getInstance().getConfig().isUseAsyncServer()) {
AsyncDataClient client =
metaGroupMember
@@ -236,7 +238,10 @@
int size = buffer.getInt();
schemas = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
- schemas.add(MeasurementSchema.deserializeFrom(buffer));
+ schemas.add(
+ buffer.get() == 0
+ ? MeasurementSchema.partialDeserializeFrom(buffer)
+ : VectorMeasurementSchema.partialDeserializeFrom(buffer));
}
}
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotPartitionTable.java b/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotPartitionTable.java
index 469e84d..bdc1578 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotPartitionTable.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/partition/slot/SlotPartitionTable.java
@@ -25,6 +25,7 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
+import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
@@ -103,7 +104,7 @@
private void init(Collection<Node> nodes) {
logger.info("Initializing a new partition table");
nodeRing.addAll(nodes);
- nodeRing.sort(Comparator.comparingInt(Node::getNodeIdentifier));
+ Collections.sort(nodeRing);
localGroups = getPartitionGroups(thisNode);
assignPartitions();
}
@@ -369,7 +370,7 @@
lastLogIndex = buffer.getLong();
nodeRing.addAll(nodeSlotMap.keySet());
- nodeRing.sort(Comparator.comparingInt(Node::getNodeIdentifier));
+ Collections.sort(nodeRing);
logger.info("All known nodes: {}", nodeRing);
localGroups = getPartitionGroups(thisNode);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPhysicalGenerator.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPhysicalGenerator.java
index a159dee..c074d92 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPhysicalGenerator.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPhysicalGenerator.java
@@ -45,6 +45,7 @@
import java.io.InputStream;
import java.net.URL;
import java.util.List;
+import java.util.Map;
import java.util.Properties;
import java.util.Set;
@@ -68,6 +69,12 @@
}
@Override
+ protected Pair<List<PartialPath>, Map<String, Integer>> getSeriesSchema(List<PartialPath> paths)
+ throws MetadataException {
+ return getCMManager().getSeriesSchemas(paths);
+ }
+
+ @Override
protected List<PartialPath> getMatchedTimeseries(PartialPath path) throws MetadataException {
return getCMManager().getMatchedPaths(path);
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanRouter.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanRouter.java
index acb0b77..5069f60 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanRouter.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanRouter.java
@@ -36,6 +36,7 @@
import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
import org.apache.iotdb.db.qp.physical.sys.AlterTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.CountPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateMultiTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.ShowChildPathsPlan;
@@ -122,6 +123,8 @@
return splitAndRoutePlan((CountPlan) plan);
} else if (plan instanceof CreateTimeSeriesPlan) {
return splitAndRoutePlan((CreateTimeSeriesPlan) plan);
+ } else if (plan instanceof CreateAlignedTimeSeriesPlan) {
+ return splitAndRoutePlan((CreateAlignedTimeSeriesPlan) plan);
} else if (plan instanceof InsertRowPlan) {
return splitAndRoutePlan((InsertRowPlan) plan);
} else if (plan instanceof AlterTimeSeriesPlan) {
@@ -160,6 +163,12 @@
return Collections.singletonMap(plan, partitionGroup);
}
+ private Map<PhysicalPlan, PartitionGroup> splitAndRoutePlan(CreateAlignedTimeSeriesPlan plan)
+ throws MetadataException {
+ PartitionGroup partitionGroup = partitionTable.partitionByPathTime(plan.getDevicePath(), 0);
+ return Collections.singletonMap(plan, partitionGroup);
+ }
+
/**
* @param plan InsertMultiTabletPlan
* @return key is InsertMultiTabletPlan, value is the partition group the plan belongs to, all
@@ -310,7 +319,7 @@
}
long[] subTimes = new long[count];
int destLoc = 0;
- Object[] values = initTabletValues(plan.getMeasurements().length, count, plan.getDataTypes());
+ Object[] values = initTabletValues(plan.getDataTypes().length, count, plan.getDataTypes());
for (int i = 0; i < locs.size(); i += 2) {
int start = locs.get(i);
int end = locs.get(i + 1);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/LocalQueryExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/LocalQueryExecutor.java
index f078519..0731ce1 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/LocalQueryExecutor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/LocalQueryExecutor.java
@@ -43,6 +43,7 @@
import org.apache.iotdb.db.exception.metadata.MetadataException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.VectorPartialPath;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
@@ -69,7 +70,7 @@
import org.apache.iotdb.tsfile.read.filter.factory.FilterFactory;
import org.apache.iotdb.tsfile.read.reader.IBatchReader;
import org.apache.iotdb.tsfile.utils.Pair;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
import com.google.common.collect.Lists;
@@ -303,7 +304,16 @@
.forEach(
fullPath -> {
try {
- paths.add(new PartialPath(fullPath));
+ if (fullPath.contains("$#$")) {
+ String[] array = fullPath.split(":");
+ List<PartialPath> subSensorsPathList = new ArrayList<>();
+ for (int i = 1; i < array.length; i++) {
+ subSensorsPathList.add(new PartialPath(array[i]));
+ }
+ paths.add(new VectorPartialPath(array[0], subSensorsPathList));
+ } else {
+ paths.add(new PartialPath(fullPath));
+ }
} catch (IllegalPathException e) {
logger.warn("Failed to create partial path, fullPath is {}.", fullPath, e);
}
@@ -440,7 +450,7 @@
// collect local timeseries schemas and send to the requester
// the measurements in them are the full paths.
List<String> prefixPaths = request.getPrefixPaths();
- List<MeasurementSchema> measurementSchemas = new ArrayList<>();
+ List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
for (String prefixPath : prefixPaths) {
getCMManager().collectSeries(new PartialPath(prefixPath), measurementSchemas);
}
@@ -459,8 +469,8 @@
DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
try {
dataOutputStream.writeInt(measurementSchemas.size());
- for (MeasurementSchema timeseriesSchema : measurementSchemas) {
- timeseriesSchema.serializeTo(dataOutputStream);
+ for (IMeasurementSchema timeseriesSchema : measurementSchemas) {
+ timeseriesSchema.partialSerializeTo(dataOutputStream);
}
} catch (IOException ignored) {
// unreachable for we are using a ByteArrayOutputStream
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
index 23603b7..bd45755 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
@@ -51,6 +51,7 @@
import org.apache.iotdb.db.exception.StorageEngineException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.VectorPartialPath;
import org.apache.iotdb.db.query.aggregation.AggregationType;
import org.apache.iotdb.db.query.context.QueryContext;
import org.apache.iotdb.db.query.control.QueryResourceManager;
@@ -62,6 +63,7 @@
import org.apache.iotdb.db.query.reader.series.SeriesRawDataPointReader;
import org.apache.iotdb.db.query.reader.series.SeriesReader;
import org.apache.iotdb.db.query.reader.series.SeriesReaderByTimestamp;
+import org.apache.iotdb.db.query.reader.series.SeriesReaderFactory;
import org.apache.iotdb.db.utils.SerializeUtils;
import org.apache.iotdb.rpc.TSStatusCode;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -513,7 +515,7 @@
((SlotPartitionTable) metaGroupMember.getPartitionTable()).getNodeSlots(header);
QueryDataSource queryDataSource =
QueryResourceManager.getInstance().getQueryDataSource(path, context, timeFilter);
- return new SeriesReader(
+ return SeriesReaderFactory.createSeriesReader(
path,
allSensors,
dataType,
@@ -662,7 +664,17 @@
List<String> fullPaths = Lists.newArrayList();
paths.forEach(
path -> {
- fullPaths.add(path.getFullPath());
+ if (path instanceof VectorPartialPath) {
+ StringBuilder builder = new StringBuilder(path.getFullPath());
+ List<PartialPath> pathList = ((VectorPartialPath) path).getSubSensorsPathList();
+ for (int i = 0; i < pathList.size(); i++) {
+ builder.append(":");
+ builder.append(pathList.get(i).getFullPath());
+ }
+ fullPaths.add(builder.toString());
+ } else {
+ fullPaths.add(path.getFullPath());
+ }
});
List<Integer> dataTypeOrdinals = Lists.newArrayList();
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/ClientServer.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/ClientServer.java
index c627373..a8c588a 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/ClientServer.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/ClientServer.java
@@ -212,6 +212,7 @@
logger.warn("Illegal plan detected: {}", plan);
return RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage());
}
+
return coordinator.executeNonQueryPlan(plan);
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/PullMeasurementSchemaHandler.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/PullMeasurementSchemaHandler.java
index aec74db..c456ef5 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/PullMeasurementSchemaHandler.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/handlers/caller/PullMeasurementSchemaHandler.java
@@ -21,7 +21,9 @@
import org.apache.iotdb.cluster.rpc.thrift.Node;
import org.apache.iotdb.cluster.rpc.thrift.PullSchemaResp;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
import org.apache.thrift.async.AsyncMethodCallback;
import org.slf4j.Logger;
@@ -38,12 +40,12 @@
private Node owner;
private List<String> prefixPaths;
- private AtomicReference<List<MeasurementSchema>> timeseriesSchemas;
+ private AtomicReference<List<IMeasurementSchema>> timeseriesSchemas;
public PullMeasurementSchemaHandler(
Node owner,
List<String> prefixPaths,
- AtomicReference<List<MeasurementSchema>> timeseriesSchemas) {
+ AtomicReference<List<IMeasurementSchema>> timeseriesSchemas) {
this.owner = owner;
this.prefixPaths = prefixPaths;
this.timeseriesSchemas = timeseriesSchemas;
@@ -53,9 +55,12 @@
public void onComplete(PullSchemaResp response) {
ByteBuffer buffer = response.schemaBytes;
int size = buffer.getInt();
- List<MeasurementSchema> schemas = new ArrayList<>(size);
+ List<IMeasurementSchema> schemas = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
- schemas.add(MeasurementSchema.deserializeFrom(buffer));
+ schemas.add(
+ buffer.get() == 0
+ ? MeasurementSchema.partialDeserializeFrom(buffer)
+ : VectorMeasurementSchema.partialDeserializeFrom(buffer));
}
synchronized (timeseriesSchemas) {
timeseriesSchemas.set(schemas);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
index ba943a9..49acf81 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/DataGroupMember.java
@@ -28,6 +28,7 @@
import org.apache.iotdb.cluster.client.sync.SyncDataHeartbeatClient;
import org.apache.iotdb.cluster.config.ClusterConstant;
import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.exception.CheckConsistencyException;
import org.apache.iotdb.cluster.exception.LogExecutionException;
import org.apache.iotdb.cluster.exception.SnapshotInstallationException;
import org.apache.iotdb.cluster.log.LogApplier;
@@ -65,6 +66,7 @@
import org.apache.iotdb.cluster.server.monitor.Peer;
import org.apache.iotdb.cluster.server.monitor.Timer;
import org.apache.iotdb.cluster.server.monitor.Timer.Statistic;
+import org.apache.iotdb.cluster.utils.IOUtils;
import org.apache.iotdb.cluster.utils.StatusUtils;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.StorageEngine;
@@ -72,6 +74,7 @@
import org.apache.iotdb.db.exception.StorageEngineException;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
import org.apache.iotdb.db.metadata.PartialPath;
import org.apache.iotdb.db.qp.executor.PlanExecutor;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
@@ -692,16 +695,38 @@
*/
@Override
public TSStatus executeNonQueryPlan(PhysicalPlan plan) {
- TSStatus status = executeNonQueryPlanWithKnownLeader(plan);
- if (!StatusUtils.NO_LEADER.equals(status)) {
- return status;
+ if (ClusterDescriptor.getInstance().getConfig().getReplicationNum() == 1) {
+ try {
+ getLocalExecutor().processNonQuery(plan);
+ return StatusUtils.OK;
+ } catch (Exception e) {
+ Throwable cause = IOUtils.getRootCause(e);
+ if (cause instanceof StorageGroupNotSetException
+ || cause instanceof UndefinedTemplateException) {
+ try {
+ metaGroupMember.syncLeaderWithConsistencyCheck(true);
+ getLocalExecutor().processNonQuery(plan);
+ return StatusUtils.OK;
+ } catch (CheckConsistencyException ce) {
+ return StatusUtils.getStatus(StatusUtils.CONSISTENCY_FAILURE, ce.getMessage());
+ } catch (Exception ne) {
+ return handleLogExecutionException(plan, IOUtils.getRootCause(ne));
+ }
+ }
+ return handleLogExecutionException(plan, cause);
+ }
+ } else {
+ TSStatus status = executeNonQueryPlanWithKnownLeader(plan);
+ if (!StatusUtils.NO_LEADER.equals(status)) {
+ return status;
+ }
+
+ long startTime = Timer.Statistic.DATA_GROUP_MEMBER_WAIT_LEADER.getOperationStartTime();
+ waitLeader();
+ Timer.Statistic.DATA_GROUP_MEMBER_WAIT_LEADER.calOperationCostTimeFromStart(startTime);
+
+ return executeNonQueryPlanWithKnownLeader(plan);
}
-
- long startTime = Timer.Statistic.DATA_GROUP_MEMBER_WAIT_LEADER.getOperationStartTime();
- waitLeader();
- Timer.Statistic.DATA_GROUP_MEMBER_WAIT_LEADER.calOperationCostTimeFromStart(startTime);
-
- return executeNonQueryPlanWithKnownLeader(plan);
}
private TSStatus executeNonQueryPlanWithKnownLeader(PhysicalPlan plan) {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
index daa1aa9..eb10d44 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/MetaGroupMember.java
@@ -88,7 +88,6 @@
import org.apache.iotdb.db.exception.metadata.MetadataException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.metadata.PartialPath;
-import org.apache.iotdb.db.qp.executor.PlanExecutor;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
import org.apache.iotdb.db.service.IoTDB;
import org.apache.iotdb.db.utils.TestOnly;
@@ -223,11 +222,6 @@
*/
private StartUpStatus startUpStatus;
- /**
- * localExecutor is used to directly execute plans like load configuration in the underlying IoTDB
- */
- private PlanExecutor localExecutor;
-
/** hardLinkCleaner will periodically clean expired hardlinks created during snapshots */
private ScheduledExecutorService hardLinkCleanerThread;
@@ -1790,13 +1784,6 @@
}
}
- public PlanExecutor getLocalExecutor() throws QueryProcessException {
- if (localExecutor == null) {
- localExecutor = new PlanExecutor();
- }
- return localExecutor;
- }
-
public StartUpStatus getStartUpStatus() {
return startUpStatus;
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/RaftMember.java b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/RaftMember.java
index 426c370..f2187f9 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/server/member/RaftMember.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/server/member/RaftMember.java
@@ -62,11 +62,14 @@
import org.apache.iotdb.cluster.utils.StatusUtils;
import org.apache.iotdb.db.exception.BatchProcessException;
import org.apache.iotdb.db.exception.IoTDBException;
+import org.apache.iotdb.db.exception.metadata.DuplicatedTemplateException;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
import org.apache.iotdb.db.exception.metadata.PathNotExistException;
import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.executor.PlanExecutor;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
import org.apache.iotdb.db.utils.TestOnly;
import org.apache.iotdb.rpc.RpcUtils;
@@ -224,6 +227,11 @@
*/
private LogDispatcher logDispatcher;
+ /**
+ * localExecutor is used to directly execute plans like load configuration in the underlying IoTDB
+ */
+ protected PlanExecutor localExecutor;
+
protected RaftMember() {}
protected RaftMember(
@@ -563,6 +571,13 @@
return response;
}
+ public PlanExecutor getLocalExecutor() throws QueryProcessException {
+ if (localExecutor == null) {
+ localExecutor = new PlanExecutor();
+ }
+ return localExecutor;
+ }
+
/**
* Get an asynchronous heartbeat thrift client to the given node.
*
@@ -972,7 +987,7 @@
return StatusUtils.OK;
}
} catch (LogExecutionException e) {
- return handleLogExecutionException(log, e);
+ return handleLogExecutionException(log.getPlan(), IOUtils.getRootCause(e));
}
return StatusUtils.TIME_OUT;
}
@@ -1036,7 +1051,7 @@
break;
}
} catch (LogExecutionException e) {
- return handleLogExecutionException(log, e);
+ return handleLogExecutionException(log.getPlan(), IOUtils.getRootCause(e));
}
return StatusUtils.TIME_OUT;
}
@@ -1468,11 +1483,18 @@
}
}
- private TSStatus handleLogExecutionException(PhysicalPlanLog log, LogExecutionException e) {
- Throwable cause = IOUtils.getRootCause(e);
+ protected TSStatus handleLogExecutionException(PhysicalPlan log, Throwable cause) {
if (cause instanceof BatchProcessException) {
return RpcUtils.getStatus(Arrays.asList(((BatchProcessException) cause).getFailingStatus()));
}
+ if (cause instanceof DuplicatedTemplateException) {
+ return StatusUtils.DUPLICATED_TEMPLATE.deepCopy().setMessage(cause.getMessage());
+ }
+ if (cause instanceof StorageGroupNotSetException) {
+ TSStatus status = StatusUtils.getStatus(TSStatusCode.STORAGE_GROUP_NOT_EXIST);
+ status.setMessage(cause.getMessage());
+ return status;
+ }
TSStatus tsStatus =
StatusUtils.getStatus(StatusUtils.EXECUTE_STATEMENT_ERROR, cause.getMessage());
if (cause instanceof RuntimeException) {
@@ -1482,7 +1504,6 @@
tsStatus.setCode(((IoTDBException) cause).getErrorCode());
}
if (!(cause instanceof PathNotExistException)
- && !(cause instanceof StorageGroupNotSetException)
&& !(cause instanceof PathAlreadyExistException)
&& !(cause instanceof StorageGroupAlreadySetException)) {
logger.debug("{} cannot be executed because ", log, cause);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/utils/PartitionUtils.java b/cluster/src/main/java/org/apache/iotdb/cluster/utils/PartitionUtils.java
index 3e583bd..9bcd9ec 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/utils/PartitionUtils.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/utils/PartitionUtils.java
@@ -23,8 +23,10 @@
import org.apache.iotdb.cluster.rpc.thrift.Node;
import org.apache.iotdb.db.engine.StorageEngine;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.qp.physical.crud.SetDeviceTemplatePlan;
import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
import org.apache.iotdb.db.qp.physical.sys.DataAuthPlan;
import org.apache.iotdb.db.qp.physical.sys.DeleteStorageGroupPlan;
@@ -97,7 +99,8 @@
|| plan instanceof AuthorPlan
|| plan instanceof DeleteStorageGroupPlan
// DataAuthPlan is global because all nodes must have all user info
- || plan instanceof DataAuthPlan;
+ || plan instanceof DataAuthPlan
+ || plan instanceof CreateTemplatePlan;
}
/**
@@ -112,7 +115,8 @@
plan instanceof DeletePlan
|| plan instanceof DeleteTimeSeriesPlan
|| plan instanceof MergePlan
- || plan instanceof FlushPlan;
+ || plan instanceof FlushPlan
+ || plan instanceof SetDeviceTemplatePlan;
}
public static int calculateStorageGroupSlotByTime(
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/utils/StatusUtils.java b/cluster/src/main/java/org/apache/iotdb/cluster/utils/StatusUtils.java
index 52231d7..719e52a 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/utils/StatusUtils.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/utils/StatusUtils.java
@@ -45,8 +45,9 @@
public static final TSStatus TIMESERIES_NOT_EXIST_ERROR =
getStatus(TSStatusCode.TIMESERIES_NOT_EXIST);
public static final TSStatus NO_CONNECTION = getStatus(TSStatusCode.NO_CONNECTION);
+ public static final TSStatus DUPLICATED_TEMPLATE = getStatus(TSStatusCode.DUPLICATED_TEMPLATE);
- private static TSStatus getStatus(TSStatusCode statusCode) {
+ public static TSStatus getStatus(TSStatusCode statusCode) {
TSStatus status = new TSStatus();
status.setCode(statusCode.getStatusCode());
switch (statusCode) {
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/client/async/AsyncClientPoolTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/client/async/AsyncClientPoolTest.java
index 6913f50..3d1d335 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/client/async/AsyncClientPoolTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/client/async/AsyncClientPoolTest.java
@@ -33,7 +33,9 @@
public class AsyncClientPoolTest {
- private final ClusterConfig config = ClusterDescriptor.getInstance().getConfig();
+ @Mock private AsyncClientFactory testAsyncClientFactory;
+
+ private ClusterConfig config = ClusterDescriptor.getInstance().getConfig();
private boolean isAsyncServer;
@Before
@@ -47,8 +49,6 @@
config.setUseAsyncServer(isAsyncServer);
}
- @Mock private AsyncClientFactory testAsyncClientFactory;
-
@Test
public void testTestClient() throws IOException {
testAsyncClientFactory = new TestAsyncClientFactory();
@@ -190,9 +190,7 @@
assertNotEquals(clients.get(0), clients.get(1));
} finally {
- ClusterDescriptor.getInstance()
- .getConfig()
- .setMaxClientPerNodePerMember(maxClientPerNodePerMember);
+ config.setMaxClientPerNodePerMember(maxClientPerNodePerMember);
}
}
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptorTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptorTest.java
index 0b37d9d..628bfd4 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptorTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/client/sync/SyncClientAdaptorTest.java
@@ -50,6 +50,7 @@
import org.apache.iotdb.db.query.context.QueryContext;
import org.apache.iotdb.service.rpc.thrift.TSStatus;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
@@ -88,7 +89,7 @@
private ByteBuffer peekNextNotNullValueResult;
private Map<Integer, SimpleSnapshot> snapshotMap;
private ByteBuffer lastResult;
- private List<MeasurementSchema> measurementSchemas;
+ private List<IMeasurementSchema> measurementSchemas;
private List<TimeseriesSchema> timeseriesSchemas;
private List<String> paths;
@@ -202,8 +203,8 @@
PullSchemaRequest request, AsyncMethodCallback<PullSchemaResp> resultHandler) {
ByteBuffer byteBuffer = ByteBuffer.allocate(4096);
byteBuffer.putInt(measurementSchemas.size());
- for (MeasurementSchema schema : measurementSchemas) {
- schema.serializeTo(byteBuffer);
+ for (IMeasurementSchema schema : measurementSchemas) {
+ schema.partialSerializeTo(byteBuffer);
}
byteBuffer.flip();
resultHandler.onComplete(new PullSchemaResp(byteBuffer));
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/common/IoTDBTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/common/IoTDBTest.java
index e7b1b7b..9f6aae3 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/common/IoTDBTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/common/IoTDBTest.java
@@ -43,7 +43,7 @@
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.expression.IExpression;
import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.junit.After;
import org.junit.Before;
@@ -133,7 +133,7 @@
private void createTimeSeries(int sgNum, int seriesNum) {
try {
- MeasurementSchema schema = TestUtils.getTestMeasurementSchema(seriesNum);
+ IMeasurementSchema schema = TestUtils.getTestMeasurementSchema(seriesNum);
planExecutor.processNonQuery(
new CreateTimeSeriesPlan(
new PartialPath(
@@ -166,7 +166,7 @@
for (String pathStr : pathStrs) {
paths.add(new PartialPath(pathStr));
}
- queryPlan.setDeduplicatedPaths(paths);
+ queryPlan.setDeduplicatedPathsAndUpdate(paths);
queryPlan.setPaths(paths);
List<TSDataType> dataTypes = new ArrayList<>();
for (PartialPath path : paths) {
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/common/TestUtils.java b/cluster/src/test/java/org/apache/iotdb/cluster/common/TestUtils.java
index d8aebfc..2ad2b46 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/common/TestUtils.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/common/TestUtils.java
@@ -53,6 +53,7 @@
import org.apache.iotdb.tsfile.write.TsFileWriter;
import org.apache.iotdb.tsfile.write.record.TSRecord;
import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
@@ -187,7 +188,7 @@
return "s" + seriesNum;
}
- public static MeasurementSchema getTestMeasurementSchema(int seriesNum) {
+ public static IMeasurementSchema getTestMeasurementSchema(int seriesNum) {
TSDataType dataType = TSDataType.DOUBLE;
TSEncoding encoding = IoTDBDescriptor.getInstance().getConfig().getDefaultDoubleEncoding();
return new MeasurementSchema(
@@ -201,7 +202,7 @@
public static MeasurementMNode getTestMeasurementMNode(int seriesNum) {
TSDataType dataType = TSDataType.DOUBLE;
TSEncoding encoding = IoTDBDescriptor.getInstance().getConfig().getDefaultDoubleEncoding();
- MeasurementSchema measurementSchema =
+ IMeasurementSchema measurementSchema =
new MeasurementSchema(
TestUtils.getTestMeasurement(seriesNum),
dataType,
@@ -387,7 +388,7 @@
file.getParentFile().mkdirs();
try (TsFileWriter writer = new TsFileWriter(file)) {
for (int k = 0; k < seriesNum; k++) {
- MeasurementSchema schema = getTestMeasurementSchema(k);
+ IMeasurementSchema schema = getTestMeasurementSchema(k);
writer.registerTimeseries(new Path(getTestSg(sgNum), schema.getMeasurementId()), schema);
}
@@ -395,7 +396,7 @@
long timestamp = i * ptNum + j;
TSRecord record = new TSRecord(timestamp, getTestSg(sgNum));
for (int k = 0; k < seriesNum; k++) {
- MeasurementSchema schema = getTestMeasurementSchema(k);
+ IMeasurementSchema schema = getTestMeasurementSchema(k);
DataPoint dataPoint =
DataPoint.getDataPoint(
schema.getType(), schema.getMeasurementId(), String.valueOf(k));
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterAggregateExecutorTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterAggregateExecutorTest.java
index db4b6fe..b76afed 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterAggregateExecutorTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterAggregateExecutorTest.java
@@ -78,7 +78,7 @@
SQLConstant.COUNT,
SQLConstant.SUM);
plan.setPaths(paths);
- plan.setDeduplicatedPaths(paths);
+ plan.setDeduplicatedPathsAndUpdate(paths);
plan.setDataTypes(dataTypes);
plan.setDeduplicatedDataTypes(dataTypes);
plan.setAggregations(aggregations);
@@ -129,7 +129,7 @@
SQLConstant.COUNT,
SQLConstant.SUM);
plan.setPaths(paths);
- plan.setDeduplicatedPaths(paths);
+ plan.setDeduplicatedPathsAndUpdate(paths);
plan.setDataTypes(dataTypes);
plan.setDeduplicatedDataTypes(dataTypes);
plan.setAggregations(aggregations);
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterDataQueryExecutorTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterDataQueryExecutorTest.java
index 8b7e98d..a303fe0 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterDataQueryExecutorTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterDataQueryExecutorTest.java
@@ -62,7 +62,7 @@
@Test
public void testNoFilter() throws IOException, StorageEngineException, QueryProcessException {
RawDataQueryPlan plan = new RawDataQueryPlan();
- plan.setDeduplicatedPaths(pathList);
+ plan.setDeduplicatedPathsAndUpdate(pathList);
plan.setDeduplicatedDataTypes(dataTypes);
queryExecutor = new ClusterDataQueryExecutor(plan, testMetaMember);
RemoteQueryContext context =
@@ -82,7 +82,7 @@
new SingleSeriesExpression(
new PartialPath(TestUtils.getTestSeries(0, 0)), ValueFilter.gtEq(5.0));
RawDataQueryPlan plan = new RawDataQueryPlan();
- plan.setDeduplicatedPaths(pathList);
+ plan.setDeduplicatedPathsAndUpdate(pathList);
plan.setDeduplicatedDataTypes(dataTypes);
plan.setExpression(expression);
queryExecutor = new ClusterDataQueryExecutor(plan, testMetaMember);
@@ -99,7 +99,7 @@
@Test
public void testNoFilterWithRedirect() throws StorageEngineException, QueryProcessException {
RawDataQueryPlan plan = new RawDataQueryPlan();
- plan.setDeduplicatedPaths(pathList);
+ plan.setDeduplicatedPathsAndUpdate(pathList);
plan.setDeduplicatedDataTypes(dataTypes);
plan.setEnableRedirect(true);
queryExecutor = new ClusterDataQueryExecutor(plan, testMetaMember);
@@ -120,7 +120,7 @@
new SingleSeriesExpression(
new PartialPath(TestUtils.getTestSeries(0, 0)), ValueFilter.gtEq(5.0));
RawDataQueryPlan plan = new RawDataQueryPlan();
- plan.setDeduplicatedPaths(pathList);
+ plan.setDeduplicatedPathsAndUpdate(pathList);
plan.setDeduplicatedDataTypes(dataTypes);
plan.setExpression(expression);
plan.setEnableRedirect(true);
@@ -141,7 +141,7 @@
IExpression expression =
new GlobalTimeExpression(new AndFilter(TimeFilter.gtEq(5), TimeFilter.ltEq(10)));
RawDataQueryPlan plan = new RawDataQueryPlan();
- plan.setDeduplicatedPaths(pathList.subList(0, 1));
+ plan.setDeduplicatedPathsAndUpdate(pathList.subList(0, 1));
plan.setDeduplicatedDataTypes(dataTypes.subList(0, 1));
plan.setExpression(expression);
plan.setEnableRedirect(true);
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterFillExecutorTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterFillExecutorTest.java
index 8061851..48e869a 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterFillExecutorTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterFillExecutorTest.java
@@ -49,7 +49,7 @@
public void testPreviousFill()
throws QueryProcessException, StorageEngineException, IOException, IllegalPathException {
FillQueryPlan plan = new FillQueryPlan();
- plan.setDeduplicatedPaths(
+ plan.setDeduplicatedPathsAndUpdate(
Collections.singletonList(new PartialPath(TestUtils.getTestSeries(0, 10))));
plan.setDeduplicatedDataTypes(Collections.singletonList(TSDataType.DOUBLE));
plan.setPaths(plan.getDeduplicatedPaths());
@@ -95,7 +95,7 @@
public void testLinearFill()
throws QueryProcessException, StorageEngineException, IOException, IllegalPathException {
FillQueryPlan plan = new FillQueryPlan();
- plan.setDeduplicatedPaths(
+ plan.setDeduplicatedPathsAndUpdate(
Collections.singletonList(new PartialPath(TestUtils.getTestSeries(0, 10))));
plan.setDeduplicatedDataTypes(Collections.singletonList(TSDataType.DOUBLE));
plan.setPaths(plan.getDeduplicatedPaths());
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterPlanExecutorTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterPlanExecutorTest.java
index d6585b9..139280b 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterPlanExecutorTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterPlanExecutorTest.java
@@ -55,7 +55,7 @@
throws QueryProcessException, QueryFilterOptimizationException, StorageEngineException,
IOException, MetadataException, InterruptedException {
RawDataQueryPlan queryPlan = new RawDataQueryPlan();
- queryPlan.setDeduplicatedPaths(pathList);
+ queryPlan.setDeduplicatedPathsAndUpdate(pathList);
queryPlan.setDeduplicatedDataTypes(dataTypes);
queryPlan.setPaths(pathList);
queryPlan.setDataTypes(dataTypes);
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterQueryRouterTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterQueryRouterTest.java
index 9cd7690..6e31c23 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterQueryRouterTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterQueryRouterTest.java
@@ -70,7 +70,7 @@
@Test
public void test() throws StorageEngineException, IOException, QueryProcessException {
RawDataQueryPlan queryPlan = new RawDataQueryPlan();
- queryPlan.setDeduplicatedPaths(pathList);
+ queryPlan.setDeduplicatedPathsAndUpdate(pathList);
queryPlan.setDeduplicatedDataTypes(dataTypes);
QueryContext context =
new RemoteQueryContext(QueryResourceManager.getInstance().assignQueryId(true, 1024, -1));
@@ -110,7 +110,7 @@
SQLConstant.COUNT,
SQLConstant.SUM);
plan.setPaths(paths);
- plan.setDeduplicatedPaths(paths);
+ plan.setDeduplicatedPathsAndUpdate(paths);
plan.setDataTypes(dataTypes);
plan.setDeduplicatedDataTypes(dataTypes);
plan.setAggregations(aggregations);
@@ -131,7 +131,7 @@
public void testPreviousFill()
throws QueryProcessException, StorageEngineException, IOException, IllegalPathException {
FillQueryPlan plan = new FillQueryPlan();
- plan.setDeduplicatedPaths(
+ plan.setDeduplicatedPathsAndUpdate(
Collections.singletonList(new PartialPath(TestUtils.getTestSeries(0, 10))));
plan.setDeduplicatedDataTypes(Collections.singletonList(TSDataType.DOUBLE));
plan.setPaths(plan.getDeduplicatedPaths());
@@ -170,7 +170,7 @@
public void testLinearFill()
throws QueryProcessException, StorageEngineException, IOException, IllegalPathException {
FillQueryPlan plan = new FillQueryPlan();
- plan.setDeduplicatedPaths(
+ plan.setDeduplicatedPathsAndUpdate(
Collections.singletonList(new PartialPath(TestUtils.getTestSeries(0, 10))));
plan.setDeduplicatedDataTypes(Collections.singletonList(TSDataType.DOUBLE));
plan.setPaths(plan.getDeduplicatedPaths());
@@ -224,7 +224,7 @@
aggregations.add(SQLConstant.COUNT);
}
groupByPlan.setPaths(pathList);
- groupByPlan.setDeduplicatedPaths(pathList);
+ groupByPlan.setDeduplicatedPathsAndUpdate(pathList);
groupByPlan.setDataTypes(dataTypes);
groupByPlan.setDeduplicatedDataTypes(dataTypes);
groupByPlan.setAggregations(aggregations);
@@ -277,7 +277,7 @@
aggregations.add(SQLConstant.COUNT);
}
groupByPlan.setPaths(pathList);
- groupByPlan.setDeduplicatedPaths(pathList);
+ groupByPlan.setDeduplicatedPathsAndUpdate(pathList);
groupByPlan.setDataTypes(dataTypes);
groupByPlan.setDeduplicatedDataTypes(dataTypes);
groupByPlan.setAggregations(aggregations);
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/query/groupby/ClusterGroupByNoVFilterDataSetTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/query/groupby/ClusterGroupByNoVFilterDataSetTest.java
index 66aa6e5..931de93 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/query/groupby/ClusterGroupByNoVFilterDataSetTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/query/groupby/ClusterGroupByNoVFilterDataSetTest.java
@@ -60,7 +60,7 @@
aggregations.add(SQLConstant.COUNT);
}
groupByPlan.setPaths(pathList);
- groupByPlan.setDeduplicatedPaths(pathList);
+ groupByPlan.setDeduplicatedPathsAndUpdate(pathList);
groupByPlan.setDataTypes(dataTypes);
groupByPlan.setDeduplicatedDataTypes(dataTypes);
groupByPlan.setAggregations(aggregations);
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/query/groupby/ClusterGroupByVFilterDataSetTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/query/groupby/ClusterGroupByVFilterDataSetTest.java
index 8e39e93..b948ff4 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/query/groupby/ClusterGroupByVFilterDataSetTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/query/groupby/ClusterGroupByVFilterDataSetTest.java
@@ -63,7 +63,7 @@
aggregations.add(SQLConstant.COUNT);
}
groupByPlan.setPaths(pathList);
- groupByPlan.setDeduplicatedPaths(pathList);
+ groupByPlan.setDeduplicatedPathsAndUpdate(pathList);
groupByPlan.setDataTypes(dataTypes);
groupByPlan.setDeduplicatedDataTypes(dataTypes);
groupByPlan.setAggregations(aggregations);
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/server/handlers/caller/PullMeasurementSchemaHandlerTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/server/handlers/caller/PullMeasurementSchemaHandlerTest.java
index bd23a0d..cb8129c 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/server/handlers/caller/PullMeasurementSchemaHandlerTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/server/handlers/caller/PullMeasurementSchemaHandlerTest.java
@@ -23,7 +23,7 @@
import org.apache.iotdb.cluster.common.TestUtils;
import org.apache.iotdb.cluster.rpc.thrift.Node;
import org.apache.iotdb.cluster.rpc.thrift.PullSchemaResp;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.junit.Test;
@@ -44,8 +44,8 @@
public void testComplete() throws InterruptedException {
Node owner = TestUtils.getNode(1);
String prefixPath = "root";
- AtomicReference<List<MeasurementSchema>> result = new AtomicReference<>();
- List<MeasurementSchema> measurementSchemas = new ArrayList<>();
+ AtomicReference<List<IMeasurementSchema>> result = new AtomicReference<>();
+ List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
for (int i = 0; i < 10; i++) {
measurementSchemas.add(TestUtils.getTestMeasurementSchema(i));
}
@@ -59,8 +59,8 @@
DataOutputStream dataOutputStream = new DataOutputStream(outputStream);
try {
dataOutputStream.writeInt(measurementSchemas.size());
- for (MeasurementSchema measurementSchema : measurementSchemas) {
- measurementSchema.serializeTo(dataOutputStream);
+ for (IMeasurementSchema measurementSchema : measurementSchemas) {
+ measurementSchema.partialSerializeTo(dataOutputStream);
}
} catch (IOException e) {
// ignore
@@ -79,7 +79,7 @@
public void testError() throws InterruptedException {
Node owner = TestUtils.getNode(1);
String prefixPath = "root";
- AtomicReference<List<MeasurementSchema>> result = new AtomicReference<>();
+ AtomicReference<List<IMeasurementSchema>> result = new AtomicReference<>();
PullMeasurementSchemaHandler handler =
new PullMeasurementSchemaHandler(owner, Collections.singletonList(prefixPath), result);
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/server/member/DataGroupMemberTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/server/member/DataGroupMemberTest.java
index 73e911d..4d092cf 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/server/member/DataGroupMemberTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/server/member/DataGroupMemberTest.java
@@ -87,7 +87,7 @@
import org.apache.iotdb.tsfile.read.filter.ValueFilter;
import org.apache.iotdb.tsfile.read.filter.basic.Filter;
import org.apache.iotdb.tsfile.read.filter.operator.AndFilter;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
import org.apache.thrift.async.AsyncMethodCallback;
@@ -672,7 +672,7 @@
PullSchemaRequest request = new PullSchemaRequest();
request.setPrefixPaths(Collections.singletonList(TestUtils.getTestSg(0)));
request.setHeader(TestUtils.getNode(0));
- AtomicReference<List<MeasurementSchema>> result = new AtomicReference<>();
+ AtomicReference<List<IMeasurementSchema>> result = new AtomicReference<>();
PullMeasurementSchemaHandler handler =
new PullMeasurementSchemaHandler(TestUtils.getNode(1), request.getPrefixPaths(), result);
new DataAsyncService(dataGroupMember).pullMeasurementSchema(request, handler);
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/server/member/MetaGroupMemberTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/server/member/MetaGroupMemberTest.java
index f98bc22..90cfeab 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/server/member/MetaGroupMemberTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/server/member/MetaGroupMemberTest.java
@@ -99,7 +99,7 @@
import org.apache.iotdb.tsfile.read.common.BatchData;
import org.apache.iotdb.tsfile.read.filter.TimeFilter;
import org.apache.iotdb.tsfile.read.filter.ValueFilter;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
import org.apache.thrift.async.AsyncMethodCallback;
@@ -278,7 +278,7 @@
}
private PullSchemaResp mockedPullTimeSeriesSchema(PullSchemaRequest request) {
- List<MeasurementSchema> schemas = new ArrayList<>();
+ List<IMeasurementSchema> schemas = new ArrayList<>();
List<String> prefixPaths = request.getPrefixPaths();
ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
@@ -287,12 +287,12 @@
if (!prefixPath.equals(TestUtils.getTestSeries(10, 0))) {
IoTDB.metaManager.collectSeries(new PartialPath(prefixPath), schemas);
dataOutputStream.writeInt(schemas.size());
- for (MeasurementSchema schema : schemas) {
- schema.serializeTo(dataOutputStream);
+ for (IMeasurementSchema schema : schemas) {
+ schema.partialSerializeTo(dataOutputStream);
}
} else {
dataOutputStream.writeInt(1);
- TestUtils.getTestMeasurementSchema(0).serializeTo(dataOutputStream);
+ TestUtils.getTestMeasurementSchema(0).partialSerializeTo(dataOutputStream);
}
}
} catch (IOException | IllegalPathException e) {
@@ -857,7 +857,7 @@
insertPlan.setDataTypes(new TSDataType[insertPlan.getMeasurements().length]);
for (int i = 0; i < 10; i++) {
insertPlan.setDeviceId(new PartialPath(TestUtils.getTestSg(i)));
- MeasurementSchema schema = TestUtils.getTestMeasurementSchema(0);
+ IMeasurementSchema schema = TestUtils.getTestMeasurementSchema(0);
try {
IoTDB.metaManager.createTimeseries(
new PartialPath(schema.getMeasurementId()),
@@ -919,7 +919,7 @@
for (int i = 0; i < 10; i++) {
insertPlan.setDeviceId(new PartialPath(TestUtils.getTestSg(i)));
- MeasurementSchema schema = TestUtils.getTestMeasurementSchema(0);
+ IMeasurementSchema schema = TestUtils.getTestMeasurementSchema(0);
try {
IoTDB.metaManager.createTimeseries(
new PartialPath(schema.getMeasurementId()),
diff --git a/cluster/src/test/resources/logback.xml b/cluster/src/test/resources/logback.xml
index 31c0be4..82ac884 100644
--- a/cluster/src/test/resources/logback.xml
+++ b/cluster/src/test/resources/logback.xml
@@ -30,7 +30,7 @@
<charset>utf-8</charset>
</encoder>
<filter class="ch.qos.logback.classic.filter.ThresholdFilter">
- <level>DEBUG</level>
+ <level>INFO</level>
</filter>
</appender>
<!--<logger name="org.apache.iotdb.db.utils.OpenFileNumUtil" level="debug" />-->
diff --git a/cluster/src/test/resources/node1conf/iotdb-cluster.properties b/cluster/src/test/resources/node1conf/iotdb-cluster.properties
index 5cb9309..c339c8f 100644
--- a/cluster/src/test/resources/node1conf/iotdb-cluster.properties
+++ b/cluster/src/test/resources/node1conf/iotdb-cluster.properties
@@ -20,7 +20,7 @@
internal_meta_port=9003
internal_data_port=40010
seed_nodes=127.0.0.1:9003,127.0.0.1:9005,127.0.0.1:9007
-default_replica_num=3
+default_replica_num=1
consistency_level=mid
connection_timeout_ms=20000
write_operation_timeout_ms=30000
diff --git a/cluster/src/test/resources/node2conf/iotdb-cluster.properties b/cluster/src/test/resources/node2conf/iotdb-cluster.properties
index 334d0f1..d287920 100644
--- a/cluster/src/test/resources/node2conf/iotdb-cluster.properties
+++ b/cluster/src/test/resources/node2conf/iotdb-cluster.properties
@@ -20,7 +20,7 @@
internal_meta_port=9005
internal_data_port=40012
seed_nodes=127.0.0.1:9003,127.0.0.1:9005,127.0.0.1:9007
-default_replica_num=3
+default_replica_num=1
consistency_level=mid
connection_timeout_ms=20000
write_operation_timeout_ms=30000
diff --git a/cluster/src/test/resources/node3conf/iotdb-cluster.properties b/cluster/src/test/resources/node3conf/iotdb-cluster.properties
index bb290f4..1bb98c4 100644
--- a/cluster/src/test/resources/node3conf/iotdb-cluster.properties
+++ b/cluster/src/test/resources/node3conf/iotdb-cluster.properties
@@ -20,7 +20,7 @@
internal_meta_port=9007
internal_data_port=40014
seed_nodes=127.0.0.1:9003,127.0.0.1:9005,127.0.0.1:9007
-default_replica_num=3
+default_replica_num=1
consistency_level=mid
connection_timeout_ms=20000
write_operation_timeout_ms=30000
diff --git a/docs/UserGuide/API/Programming-Java-Native-API.md b/docs/UserGuide/API/Programming-Java-Native-API.md
index 1586ff7..686e956 100644
--- a/docs/UserGuide/API/Programming-Java-Native-API.md
+++ b/docs/UserGuide/API/Programming-Java-Native-API.md
@@ -101,6 +101,15 @@
List<Map<String, String>> attributesList, List<String> measurementAliasList)
```
+* Create aligned timeseries
+```
+void createAlignedTimeseries(String devicePath, List<String> measurements,
+ List<TSDataType> dataTypes, List<TSEncoding> encodings,
+ CompressionType compressor, List<String> measurementAliasList);
+```
+
+Attention: Alias of measurements are **not supported** currently.
+
* Delete one or several timeseries
```java
@@ -260,6 +269,8 @@
Or `example/session/src/main/java/org/apache/iotdb/SessionPoolExample.java`
+For examples of aligned timeseries and device template, you can refer to `example/session/src/main/java/org/apache/iotdb/VectorSessionExample.java`
+
@@ -335,6 +346,17 @@
Create multiple timeseries with a single method. Users can provide props, tags, attributes and measurementAlias as well for detailed timeseries information.
```java
+void createAlignedTimeseries(String devicePath, List<String> measurements,
+ List<TSDataType> dataTypes, List<TSEncoding> encodings,
+ CompressionType compressor, List<String> measurementAliasList);
+```
+
+Create aligned timeseries with device path, measurements, data types, encodings, compression.
+
+Attention: Alias of measurements are **not supported** currently.
+
+```java
+
boolean checkTimeseriesExists(String path)
```
@@ -347,3 +369,40 @@
Open a session and specifies whether the Leader cache is enabled. Note that this interface improves performance for distributed IoTDB, but adds less cost to the client for stand-alone IoTDB.
+```
+
+* name: template name
+* measurements: List of measurements, if it is a single measurement, just put it's name
+* into a list and add to measurements if it is a vector measurement, put all measurements of
+* the vector into a list and add to measurements
+* dataTypes: List of datatypes, if it is a single measurement, just put it's type into a
+* list and add to dataTypes if it is a vector measurement, put all types of the vector
+* into a list and add to dataTypes
+* encodings: List of encodings, if it is a single measurement, just put it's encoding into
+* a list and add to encodings if it is a vector measurement, put all encodings of the
+* vector into a list and add to encodings
+* compressors: List of compressors
+void createDeviceTemplate(
+ String name,
+ List<List<String>> measurements,
+ List<List<TSDataType>> dataTypes,
+ List<List<TSEncoding>> encodings,
+ List<CompressionType> compressors)
+```
+
+Create a device template, the param description at above
+
+```
+
+void setDeviceTemplate(String templateName, String prefixPath)
+
+```
+
+Set the device template named 'templateName' at path 'prefixPath'. You should firstly create the template using
+
+```
+
+void createDeviceTemplate
+
+```
+
diff --git a/docs/UserGuide/Appendix/SQL-Reference.md b/docs/UserGuide/Appendix/SQL-Reference.md
index 8f88ea4..442b8dd 100644
--- a/docs/UserGuide/Appendix/SQL-Reference.md
+++ b/docs/UserGuide/Appendix/SQL-Reference.md
@@ -104,6 +104,27 @@
Note: For SDT, it is optional to set compression maximum COMPMAXTIME, which is the maximum time difference between stored values regardless of COMPDEV.
```
+* Create device template
+```
+CREATE device template <TemplateName> WITH <AttributeClauses>
+attributeClauses
+ : (MEASUREMENT_NAME DATATYPE OPERATOR_EQ dataType COMMA ENCODING OPERATOR_EQ encoding
+ (COMMA (COMPRESSOR | COMPRESSION) OPERATOR_EQ compressor=propertyValue)?
+ (COMMA property)*)
+ attributeClause
+ ;
+Eg: create device template temp1(
+ (s1 INT32 with encoding=Gorilla, compression=SNAPPY),
+ (s2 FLOAT with encoding=RLE, compression=SNAPPY)
+ )
+```
+
+* Set device template
+```
+set device template <TemplateName> to <STORAGE_GROUP_NAME>
+Eg: set device template temp1 to root.beijing
+```
+
* Delete Timeseries Statement
```
@@ -303,12 +324,13 @@
* Insert Record Statement
```
-INSERT INTO <PrefixPath> LPAREN TIMESTAMP COMMA <Sensor> [COMMA <Sensor>]* RPAREN VALUES LPAREN <TimeValue>, <PointValue> [COMMA <PointValue>]* RPAREN
-Sensor : Identifier
+INSERT INTO <PrefixPath> LPAREN TIMESTAMP COMMA <MeasurementName> [COMMA <MeasurementName>]* RPAREN VALUES LPAREN <TimeValue>, <PointValue> [COMMA <PointValue>]* RPAREN
+MeasurementName : Identifier | LPAREN Identifier (COMMA Identifier)+ RPAREN
Eg: IoTDB > INSERT INTO root.ln.wf01.wt01(timestamp,status) values(1509465600000,true)
Eg: IoTDB > INSERT INTO root.ln.wf01.wt01(timestamp,status) VALUES(NOW(), false)
Eg: IoTDB > INSERT INTO root.ln.wf01.wt01(timestamp,temperature) VALUES(2017-11-01T00:17:00.000+08:00,24.22028)
-Eg: IoTDB > INSERT INTO root.ln.wf01.wt01(timestamp, status, temperature) VALUES (1509466680000, false, 20.060787);
+Eg: IoTDB > INSERT INTO root.ln.wf01.wt01(timestamp,status,temperature) VALUES (1509466680000,false,20.060787)
+Eg: IoTDB > INSERT INTO root.sg.d1(timestamp,(s1,s2),(s3,s4)) VALUES (1509466680000,(1.0,2),(NULL,4))
Note: the statement needs to satisfy this constraint: <PrefixPath> + <Path> = <Timeseries>
Note: The order of Sensor and PointValue need one-to-one correspondence
```
diff --git a/docs/UserGuide/Data-Concept/Data-Model-and-Terminology.md b/docs/UserGuide/Data-Concept/Data-Model-and-Terminology.md
index 479b9c7..98043ae 100644
--- a/docs/UserGuide/Data-Concept/Data-Model-and-Terminology.md
+++ b/docs/UserGuide/Data-Concept/Data-Model-and-Terminology.md
@@ -101,6 +101,29 @@
> Note: The layer of timeseries paths supported by the current IoTDB must be greater than or equal to four (it will be changed to two in the future).
+
+* Aligned timeseries (From v0.13)
+
+When a group of sensors detects data at the same time, multiple timeseries with the same timestamp will be produced, which are called **aligned timeseries** in IoTDB (and are also called **multivariate timeseries** academically. It contains multiple unary timeseries as components, and the sampling time of each unary timeseries is the same.)
+
+Aligned timeseries can be created, inserted values, and deleted at the same time. However, when querying, each sensor can be queried separately.
+
+By using aligned timeseries, the timestamp column could be stored only once in memory and disk when inserting data, instead of stored as many times as the number of timeseries:
+
+<img style="width:100%; max-width:800px; max-height:600px; margin-left:auto; margin-right:auto; display:block;" src="https://user-images.githubusercontent.com/19167280/114125919-f4850800-9929-11eb-8211-81d4c04af1ec.png">
+
+In the following chapters of data definition language, data operation language and Java Native Interface, various operations related to aligned timeseries will be introduced one by one.
+
+
+* Device template (From v0.13)
+
+In the actual scenario, there are many devices with the same model, that is, they have the same working condition name and type. To save system resources, you can declare a **device template** for the same type of device, mount it to any node in the path.
+
+Currently you can only set one **device template** on a specific path. Device will use it's own device template or nearest ancestor's device template.
+
+In the following chapters of data definition language, data operation language and Java Native Interface, various operations related to device template will be introduced one by one.
+
+
* Prefix Path
The prefix path refers to the path where the prefix of a timeseries path is located. A prefix path contains all timeseries paths prefixed by the path. For example, suppose that we have three sensors: `root.vehicle.device1.sensor1`, `root.vehicle.device1.sensor2`, `root.vehicle.device2.sensor1`, the prefix path `root.vehicle.device1` contains two timeseries paths `root.vehicle.device1.sensor1` and `root.vehicle.device1.sensor2` while `root.vehicle.device2.sensor1` is excluded.
@@ -237,4 +260,3 @@
now() - 1w //1 week earlier than the current server time
```
> Note:There must be spaces on the left and right of '+' and '-'.
-
diff --git a/docs/UserGuide/IoTDB-SQL-Language/DDL-Data-Definition-Language.md b/docs/UserGuide/IoTDB-SQL-Language/DDL-Data-Definition-Language.md
index c91cc25..33622ec 100644
--- a/docs/UserGuide/IoTDB-SQL-Language/DDL-Data-Definition-Language.md
+++ b/docs/UserGuide/IoTDB-SQL-Language/DDL-Data-Definition-Language.md
@@ -91,15 +91,40 @@
IoTDB > create timeseries root.sgcc.wf03.wt01.temperature with datatype=FLOAT,encoding=RLE
```
-Notice that when in the CRATE TIMESERIES statement the encoding method conflicts with the data type, the system gives the corresponding error prompt as shown below:
+We could also create **aligned** timeseries:
```
-IoTDB> create timeseries root.ln.wf02.wt02.status WITH DATATYPE=BOOLEAN, ENCODING=TS_2DIFF
+IoTDB > create aligned timeseries root.sg.d1.(s1 FLOAT, s2 INT32)
+IoTDB > create aligned timeseries root.sg.d1.(s3 FLOAT, s4 INT32) with encoding=(RLE, Grollia), compression=SNAPPY
+```
+
+Attention: Aligned timeseries must have the same compression type.
+
+Notice that when in the CREATE TIMESERIES statement the encoding method conflicts with the data type, the system gives the corresponding error prompt as shown below:
+
+```
+IoTDB > create timeseries root.ln.wf02.wt02.status WITH DATATYPE=BOOLEAN, ENCODING=TS_2DIFF
error: encoding TS_2DIFF does not support BOOLEAN
```
Please refer to [Encoding](../Data-Concept/Encoding.md) for correspondence between data type and encoding.
+### Create and set device template
+```
+
+IoTDB > set storage group root.beijing
+
+// create a device templat
+IoTDB > create device template temp1(
+ (s1 INT32 with encoding=Gorilla, compression=SNAPPY),
+ (s2 FLOAT with encoding=RLE, compression=SNAPPY)
+ )
+
+// set device template to storage group "root.beijing"
+IoTDB > set device template temp1 to root.beijing
+
+```
+
#### Delete Timeseries
To delete the timeseries we created before, we are able to use `DELETE TimeSeries <PrefixPath>` statement.
@@ -112,6 +137,19 @@
IoTDB> delete timeseries root.ln.wf02.*
```
+As for **aligned** timeseries, we could delete them by explicit declaration with parentheses.
+
+```
+IoTDB > delete timeseries root.sg.d1.(s1,s2)
+```
+
+Attention: Deleting part of aligned timeseries is **not supported** currently.
+
+```
+IoTDB > delete timeseries root.sg.d1.s1
+error: Not support deleting part of aligned timeseies!
+```
+
#### Show Timeseries
* SHOW LATEST? TIMESERIES prefixPath? showWhereClause? limitClause?
diff --git a/docs/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md b/docs/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
index e155237..60e44ac 100644
--- a/docs/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
+++ b/docs/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
@@ -68,6 +68,13 @@
It costs 0.170s
```
+As for **aligned** timeseries,we could insert values into measurements by **explicit** declaration with parentheses. Empty values could be represented by `NULL` or `null`:
+
+```
+IoTDB > insert into root.sg.d1(timestamp,(s1,s2),(s3,s4)) values (1509466680000,(1.0,2),(null,4))
+IoTDB > insert into root.sg.d1(timestamp,(s1,s2)) values (1509466680001,(NULL,1))
+```
+
## SELECT
### Time Slice Query
diff --git a/docs/zh/UserGuide/API/Programming-Java-Native-API.md b/docs/zh/UserGuide/API/Programming-Java-Native-API.md
index 23512ca..694c6c1 100644
--- a/docs/zh/UserGuide/API/Programming-Java-Native-API.md
+++ b/docs/zh/UserGuide/API/Programming-Java-Native-API.md
@@ -102,6 +102,16 @@
List<Map<String, String>> attributesList, List<String> measurementAliasList)
```
+* 创建对齐时间序列
+
+```
+void createAlignedTimeseries(String devicePath, List<String> measurements,
+ List<TSDataType> dataTypes, List<TSEncoding> encodings,
+ CompressionType compressor, List<String> measurementAliasList);
+```
+
+注意:目前**暂不支持**使用传感器别名。
+
* 删除一个或多个时间序列
```java
@@ -182,6 +192,33 @@
void executeNonQueryStatement(String sql)
```
+* 创建一个设备模板
+
+```
+* name: 设备模板名称
+* measurements: 工况名称列表,如果该工况是非对齐的,直接将其名称放入一个list中再放入measurements中,
+* 如果该工况是对齐的,将所有对齐工况名称放入一个list再放入measurements中
+* dataTypes: 数据类型名称列表,如果该工况是非对齐的,直接将其数据类型放入一个list中再放入dataTypes中,
+ 如果该工况是对齐的,将所有对齐工况的数据类型放入一个list再放入dataTypes中
+* encodings: 编码类型名称列表,如果该工况是非对齐的,直接将其数据类型放入一个list中再放入encodings中,
+ 如果该工况是对齐的,将所有对齐工况的编码类型放入一个list再放入encodings中
+* compressors: 压缩方式列表
+void createDeviceTemplate(
+ String name,
+ List<List<String>> measurements,
+ List<List<TSDataType>> dataTypes,
+ List<List<TSEncoding>> encodings,
+ List<CompressionType> compressors)
+```
+
+
+* 将名为'templateName'的设备模板挂载到'prefixPath'路径下,在执行这一步之前,你需要创建名为'templateName'的设备模板
+
+```
+void setDeviceTemplate(String templateName, String prefixPath)
+```
+
+
### 测试接口说明
@@ -239,7 +276,9 @@
或 `example/session/src/main/java/org/apache/iotdb/SessionPoolExample.java`
+使用对齐时间序列和设备模板的示例可以参见 `example/session/src/main/java/org/apache/iotdb/VectorSessionExample.java`。
+
### 示例代码
diff --git a/docs/zh/UserGuide/Appendix/SQL-Reference.md b/docs/zh/UserGuide/Appendix/SQL-Reference.md
index e574bfb..9d5e59e 100644
--- a/docs/zh/UserGuide/Appendix/SQL-Reference.md
+++ b/docs/zh/UserGuide/Appendix/SQL-Reference.md
@@ -94,6 +94,27 @@
Note: For SDT, it is optional to set compression maximum COMPMAXTIME, which is the maximum time difference between stored values regardless of COMPDEV.
```
+* 创建设备模板语句
+```
+CREATE device template <TemplateName> WITH <AttributeClauses>
+attributeClauses
+ : (MEASUREMENT_NAME DATATYPE OPERATOR_EQ dataType COMMA ENCODING OPERATOR_EQ encoding
+ (COMMA (COMPRESSOR | COMPRESSION) OPERATOR_EQ compressor=propertyValue)?
+ (COMMA property)*)
+ attributeClause
+ ;
+Eg: create device template temp1(
+ (s1 INT32 with encoding=Gorilla, compression=SNAPPY),
+ (s2 FLOAT with encoding=RLE, compression=SNAPPY)
+ )
+```
+
+* 挂载设备模板语句
+```
+set device template <TemplateName> to <STORAGE_GROUP_NAME>
+Eg: set device template temp1 to root.beijing
+```
+
* 删除时间序列语句
```
@@ -297,12 +318,13 @@
* 插入记录语句
```
-INSERT INTO <PrefixPath> LPAREN TIMESTAMP COMMA <Sensor> [COMMA <Sensor>]* RPAREN VALUES LPAREN <TimeValue>, <PointValue> [COMMA <PointValue>]* RPAREN
-Sensor : Identifier
+INSERT INTO <PrefixPath> LPAREN TIMESTAMP COMMA <MeasurementName> [COMMA <MeasurementName>]* RPAREN VALUES LPAREN <TimeValue>, <PointValue> [COMMA <PointValue>]* RPAREN
+MeasurementName : Identifier | LPAREN Identifier (COMMA Identifier)+ RPAREN
Eg: IoTDB > INSERT INTO root.ln.wf01.wt01(timestamp,status) values(1509465600000,true)
Eg: IoTDB > INSERT INTO root.ln.wf01.wt01(timestamp,status) VALUES(NOW(), false)
Eg: IoTDB > INSERT INTO root.ln.wf01.wt01(timestamp,temperature) VALUES(2017-11-01T00:17:00.000+08:00,24.22028)
-Eg: IoTDB > INSERT INTO root.ln.wf01.wt01(timestamp, status, temperature) VALUES (1509466680000, false, 20.060787);
+Eg: IoTDB > INSERT INTO root.ln.wf01.wt01(timestamp,status,temperature) VALUES (1509466680000,false,20.060787)
+Eg: IoTDB > INSERT INTO root.sg.d1(timestamp,(s1,s2),(s3,s4)) VALUES (1509466680000,(1.0,2),(NULL,4))
Note: the statement needs to satisfy this constraint: <PrefixPath> + <Path> = <Timeseries>
Note: The order of Sensor and PointValue need one-to-one correspondence
```
diff --git a/docs/zh/UserGuide/Data-Concept/Data-Model-and-Terminology.md b/docs/zh/UserGuide/Data-Concept/Data-Model-and-Terminology.md
index a095ad1..ec7f533 100644
--- a/docs/zh/UserGuide/Data-Concept/Data-Model-and-Terminology.md
+++ b/docs/zh/UserGuide/Data-Concept/Data-Model-and-Terminology.md
@@ -97,11 +97,32 @@
> 注意:当前IoTDB支持的时间序列必须大于等于四层(之后会更改为两层)。
+* 对齐时间序列(v0.13 起支持)
+
+在同一个时间戳有多个传感器同时采样,会形成具有相同时间戳的多条时间序列,在 IoTDB 中,这些时间序列成为**对齐时间序列**(在学术上也称为**多元时间序列**,即包含多个一元时间序列作为分量, 各个一元时间序列的采样时间点相同)。
+
+对齐时间序列可以被同时创建,同时插入值,删除时也必须同时删除。不过在查询的时候,可以对于每一个传感器单独查询。
+
+通过使用对齐的时间序列,在插入数据时,一组对齐序列的时间戳列在内存和磁盘中仅需存储一次,而不是时间序列的条数次:
+
+<img style="width:100%; max-width:800px; max-height:600px; margin-left:auto; margin-right:auto; display:block;" src="https://user-images.githubusercontent.com/19167280/114125919-f4850800-9929-11eb-8211-81d4c04af1ec.png">
+
+在后续数据定义语言、数据操作语言和 Java 原生接口章节,将对涉及到对齐时间序列的各种操作进行逐一介绍。
+
+
+* 设备模板(v0.13 起支持)
+
+实际场景中有许多设备型号相同,即具有相同的工况名称和类型,为了节省系统资源,可以声明一个**设备模板**表征同一类型的设备,设备模版可以挂在到路径的任意节点上。
+
+目前每一个路径节点仅允许挂载一个设备模板,具体的设备将使用其自身或最近祖先的设备模板作为有效模板。
+
+在后续数据定义语言、数据操作语言和 Java 原生接口章节,将对涉及到设备模板的各种操作进行逐一介绍。
+
* 前缀路径
前缀路径是指一个时间序列的前缀所在的路径,一个前缀路径包含以该路径为前缀的所有时间序列。例如当前我们有`root.vehicle.device1.sensor1`, `root.vehicle.device1.sensor2`, `root.vehicle.device2.sensor1`三个传感器,则`root.vehicle.device1`前缀路径包含`root.vehicle.device1.sensor1`、`root.vehicle.device1.sensor2`两个时间序列,而不包含`root.vehicle.device2.sensor1`。
-* 3.1.7 带`*`路径
+* 带`*`路径
为了使得在表达多个时间序列或表达前缀路径的时候更加方便快捷,IoTDB为用户提供带`*`路径。`*`可以出现在路径中的任何层。按照`*`出现的位置,带`*`路径可以分为两种:
`*`出现在路径的结尾;
@@ -233,4 +254,3 @@
now() - 1w //比服务器时间早1周的时间
```
> 注意:'+'和'-'的左右两边必须有空格
-
diff --git a/docs/zh/UserGuide/IoTDB-SQL-Language/DDL-Data-Definition-Language.md b/docs/zh/UserGuide/IoTDB-SQL-Language/DDL-Data-Definition-Language.md
index a01db82..1ec7a865 100644
--- a/docs/zh/UserGuide/IoTDB-SQL-Language/DDL-Data-Definition-Language.md
+++ b/docs/zh/UserGuide/IoTDB-SQL-Language/DDL-Data-Definition-Language.md
@@ -93,6 +93,15 @@
IoTDB > create timeseries root.sgcc.wf03.wt01.temperature with datatype=FLOAT,encoding=RLE
```
+我们也可以创建**对齐**时间序列:
+
+```
+IoTDB > create aligned timeseries root.sg.d1.(s1 FLOAT, s2 INT32)
+IoTDB > create aligned timeseries root.sg.d1.(s3 FLOAT, s4 INT32) with encoding=(RLE, Grollia), compression=SNAPPY
+```
+
+注意:对齐时间序列必须拥有相同的压缩方式。
+
需要注意的是,当创建时间序列时指定的编码方式与数据类型不对应时,系统会给出相应的错误提示,如下所示:
```
IoTDB> create timeseries root.ln.wf02.wt02.status WITH DATATYPE=BOOLEAN, ENCODING=TS_2DIFF
@@ -101,6 +110,22 @@
详细的数据类型与编码方式的对应列表请参见[编码方式](../Data-Concept/Encoding.md)。
+### 创建和挂载设备模板
+```
+
+IoTDB > set storage group root.beijing
+
+// 创建设备模板
+IoTDB > create device template temp1(
+ (s1 INT32 with encoding=Gorilla, compression=SNAPPY),
+ (s2 FLOAT with encoding=RLE, compression=SNAPPY)
+ )
+
+// 将设备模板挂载到root.beijing存储组上
+IoTDB > set device template temp1 to root.beijing
+
+```
+
#### 删除时间序列
我们可以使用`DELETE TimeSeries <PrefixPath>`语句来删除我们之前创建的时间序列。SQL语句如下所示:
@@ -111,6 +136,20 @@
IoTDB> delete timeseries root.ln.wf02.*
```
+对于**对齐**时间序列,我们可以通过括号来显式地删除整组序列:
+
+```
+IoTDB > delete timeseries root.sg.d1.(s1,s2)
+```
+
+注意:目前暂不支持删除部分对齐时间序列。
+
+```
+IoTDB > delete timeseries root.sg.d1.s1
+error: Not support deleting part of aligned timeseies!
+```
+
+
#### 查看时间序列
* SHOW LATEST? TIMESERIES prefixPath? showWhereClause? limitClause?
diff --git a/docs/zh/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md b/docs/zh/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
index adfe72e..c25e204 100644
--- a/docs/zh/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
+++ b/docs/zh/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
@@ -47,7 +47,7 @@
INSERT语句还可以支持在同一个时间点下多列数据的插入,同时向2时间点插入上述两个时间序列的值,多列数据插入示例代码如下:
```
-IoTDB > insert into root.ln.wf02.wt02(timestamp, status, hardware) VALUES (2, false, 'v2')
+IoTDB > insert into root.ln.wf02.wt02(timestamp, status, hardware) values (2, false, 'v2')
```
插入数据后我们可以使用SELECT语句简单查询已插入的数据。
@@ -69,6 +69,13 @@
It costs 0.170s
```
+对于**对齐**时间序列而言,可以通过括号来**显式地**同时向序列中所有传感器插入值。其中,空值可以用 `NULL` 或者 `null` 占用:
+
+```
+IoTDB > insert into root.sg.d1(timestamp,(s1,s2),(s3,s4)) values (1509466680000,(1.0,2),(null,4))
+IoTDB > insert into root.sg.d1(timestamp,(s1,s2)) values (1509466680001,(NULL,1))
+```
+
### 数据查询
#### 时间条件过滤查询
diff --git a/example/hadoop/src/main/java/org/apache/iotdb/hadoop/tsfile/TsFileHelper.java b/example/hadoop/src/main/java/org/apache/iotdb/hadoop/tsfile/TsFileHelper.java
index b514a73..0c07677 100644
--- a/example/hadoop/src/main/java/org/apache/iotdb/hadoop/tsfile/TsFileHelper.java
+++ b/example/hadoop/src/main/java/org/apache/iotdb/hadoop/tsfile/TsFileHelper.java
@@ -24,6 +24,7 @@
import org.apache.iotdb.tsfile.read.common.Path;
import org.apache.iotdb.tsfile.write.TsFileWriter;
import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.Schema;
@@ -55,7 +56,7 @@
Schema schema = new Schema();
- List<MeasurementSchema> schemaList = new ArrayList<>();
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
// the number of rows to include in the tablet
int rowNum = 1000000;
@@ -73,7 +74,7 @@
}
for (int i = 2; i < sensorNum; i++) {
- MeasurementSchema measurementSchema =
+ IMeasurementSchema measurementSchema =
new MeasurementSchema(
Constant.SENSOR_PREFIX + (i + 1), TSDataType.DOUBLE, TSEncoding.TS_2DIFF);
schema.registerTimeseries(
diff --git a/example/session/src/main/java/org/apache/iotdb/DataMigrationExample.java b/example/session/src/main/java/org/apache/iotdb/DataMigrationExample.java
index 4bfacfc..4ef8b10 100644
--- a/example/session/src/main/java/org/apache/iotdb/DataMigrationExample.java
+++ b/example/session/src/main/java/org/apache/iotdb/DataMigrationExample.java
@@ -27,6 +27,7 @@
import org.apache.iotdb.tsfile.read.common.Path;
import org.apache.iotdb.tsfile.utils.Binary;
import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import java.util.ArrayList;
@@ -121,7 +122,7 @@
this.measurement = series.getMeasurement();
this.dataType = dataType;
this.series = series;
- List<MeasurementSchema> schemaList = new ArrayList<>();
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
schemaList.add(new MeasurementSchema(measurement, dataType));
tablet = new Tablet(device, schemaList, 300000);
}
diff --git a/example/session/src/main/java/org/apache/iotdb/SessionExample.java b/example/session/src/main/java/org/apache/iotdb/SessionExample.java
index 386b4e4..d996d10 100644
--- a/example/session/src/main/java/org/apache/iotdb/SessionExample.java
+++ b/example/session/src/main/java/org/apache/iotdb/SessionExample.java
@@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb;
import org.apache.iotdb.rpc.IoTDBConnectionException;
@@ -28,6 +29,7 @@
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import java.util.ArrayList;
@@ -321,7 +323,7 @@
*/
// The schema of measurements of one device
// only measurementId and data type in MeasurementSchema take effects in Tablet
- List<MeasurementSchema> schemaList = new ArrayList<>();
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
schemaList.add(new MeasurementSchema("s1", TSDataType.INT64));
schemaList.add(new MeasurementSchema("s2", TSDataType.INT64));
schemaList.add(new MeasurementSchema("s3", TSDataType.INT64));
@@ -376,7 +378,7 @@
private static void insertTablets() throws IoTDBConnectionException, StatementExecutionException {
// The schema of measurements of one device
// only measurementId and data type in MeasurementSchema take effects in Tablet
- List<MeasurementSchema> schemaList = new ArrayList<>();
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
schemaList.add(new MeasurementSchema("s1", TSDataType.INT64));
schemaList.add(new MeasurementSchema("s2", TSDataType.INT64));
schemaList.add(new MeasurementSchema("s3", TSDataType.INT64));
diff --git a/example/session/src/main/java/org/apache/iotdb/VectorSessionExample.java b/example/session/src/main/java/org/apache/iotdb/VectorSessionExample.java
new file mode 100644
index 0000000..afeef91
--- /dev/null
+++ b/example/session/src/main/java/org/apache/iotdb/VectorSessionExample.java
@@ -0,0 +1,337 @@
+/*
+ * 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;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.session.Session;
+import org.apache.iotdb.session.SessionDataSet;
+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.utils.BitMap;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
+
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.List;
+
+@SuppressWarnings("squid:S106")
+public class VectorSessionExample {
+
+ private static Session session;
+ private static final String ROOT_SG1_D1 = "root.sg_1.d1";
+
+ public static void main(String[] args)
+ throws IoTDBConnectionException, StatementExecutionException {
+ session = new Session("127.0.0.1", 6667, "root", "root");
+ session.open(false);
+
+ // set session fetchSize
+ session.setFetchSize(10000);
+
+ createTemplate();
+ insertTabletWithAlignedTimeseriesMethod1();
+ insertTabletWithAlignedTimeseriesMethod2();
+
+ insertNullableTabletWithAlignedTimeseries();
+ selectTest();
+ selectWithValueFilterTest();
+
+ selectWithGroupByTest();
+ selectWithLastTest();
+
+ selectWithAggregationTest();
+
+ selectWithAlignByDeviceTest();
+
+ session.close();
+ }
+
+ private static void selectTest() throws StatementExecutionException, IoTDBConnectionException {
+ SessionDataSet dataSet = session.executeQueryStatement("select s1 from root.sg_1.d1");
+ System.out.println(dataSet.getColumnNames());
+ while (dataSet.hasNext()) {
+ System.out.println(dataSet.next());
+ }
+
+ dataSet.closeOperationHandle();
+ dataSet = session.executeQueryStatement("select * from root.sg_1.d1");
+ System.out.println(dataSet.getColumnNames());
+ while (dataSet.hasNext()) {
+ System.out.println(dataSet.next());
+ }
+
+ dataSet.closeOperationHandle();
+ }
+
+ private static void selectWithAlignByDeviceTest()
+ throws StatementExecutionException, IoTDBConnectionException {
+ SessionDataSet dataSet =
+ session.executeQueryStatement("select * from root.sg_1 align by device");
+ System.out.println(dataSet.getColumnNames());
+ while (dataSet.hasNext()) {
+ System.out.println(dataSet.next());
+ }
+
+ dataSet.closeOperationHandle();
+ }
+
+ private static void selectWithValueFilterTest()
+ throws StatementExecutionException, IoTDBConnectionException {
+ SessionDataSet dataSet =
+ session.executeQueryStatement("select s1 from root.sg_1.d1 where s1 > 0");
+ System.out.println(dataSet.getColumnNames());
+ while (dataSet.hasNext()) {
+ System.out.println(dataSet.next());
+ }
+
+ dataSet.closeOperationHandle();
+ dataSet =
+ session.executeQueryStatement(
+ "select * from root.sg_1.d1 where time > 50 and s1 > 0 and s2 > 10000");
+ System.out.println(dataSet.getColumnNames());
+ while (dataSet.hasNext()) {
+ System.out.println(dataSet.next());
+ }
+
+ dataSet.closeOperationHandle();
+ }
+
+ private static void selectWithAggregationTest()
+ throws StatementExecutionException, IoTDBConnectionException {
+ SessionDataSet dataSet = session.executeQueryStatement("select count(s1) from root.sg_1.d1");
+ System.out.println(dataSet.getColumnNames());
+ while (dataSet.hasNext()) {
+ System.out.println(dataSet.next());
+ }
+
+ dataSet.closeOperationHandle();
+ dataSet =
+ session.executeQueryStatement(
+ "select sum(*) from root.sg_1.d1 where time > 50 and s1 > 0 and s2 > 10000");
+ System.out.println(dataSet.getColumnNames());
+ while (dataSet.hasNext()) {
+ System.out.println(dataSet.next());
+ }
+
+ dataSet.closeOperationHandle();
+ }
+
+ private static void selectWithGroupByTest()
+ throws StatementExecutionException, IoTDBConnectionException {
+ SessionDataSet dataSet =
+ session.executeQueryStatement(
+ "select count(s1) from root.sg_1.d1 GROUP BY ([1, 100), 20ms)");
+ System.out.println(dataSet.getColumnNames());
+ while (dataSet.hasNext()) {
+ System.out.println(dataSet.next());
+ }
+
+ dataSet.closeOperationHandle();
+ dataSet =
+ session.executeQueryStatement(
+ "select count(*) from root.sg_1.d1 where time > 50 and s1 > 0 and s2 > 10000"
+ + " GROUP BY ([50, 100), 10ms)");
+ System.out.println(dataSet.getColumnNames());
+ while (dataSet.hasNext()) {
+ System.out.println(dataSet.next());
+ }
+
+ dataSet.closeOperationHandle();
+ }
+
+ private static void selectWithLastTest()
+ throws StatementExecutionException, IoTDBConnectionException {
+ SessionDataSet dataSet = session.executeQueryStatement("select last s1 from root.sg_1.d1");
+ System.out.println(dataSet.getColumnNames());
+ while (dataSet.hasNext()) {
+ System.out.println(dataSet.next());
+ }
+
+ dataSet.closeOperationHandle();
+ dataSet = session.executeQueryStatement("select last * from root.sg_1.d1");
+ System.out.println(dataSet.getColumnNames());
+ while (dataSet.hasNext()) {
+ System.out.println(dataSet.next());
+ }
+
+ dataSet.closeOperationHandle();
+ }
+
+ // be sure template is coordinate with tablet
+ private static void createTemplate()
+ throws StatementExecutionException, IoTDBConnectionException {
+ List<List<String>> measurementList = new ArrayList<>();
+ List<String> vectorMeasurement = new ArrayList<>();
+ for (int i = 1; i <= 2; i++) {
+ vectorMeasurement.add("s" + i);
+ }
+ measurementList.add(vectorMeasurement);
+
+ List<List<TSDataType>> dataTypeList = new ArrayList<>();
+ List<TSDataType> vectorDatatype = new ArrayList<>();
+ vectorDatatype.add(TSDataType.INT64);
+ vectorDatatype.add(TSDataType.INT32);
+ dataTypeList.add(vectorDatatype);
+
+ List<List<TSEncoding>> encodingList = new ArrayList<>();
+ List<TSEncoding> vectorEncoding = new ArrayList<>();
+ for (int i = 1; i <= 2; i++) {
+ vectorEncoding.add(TSEncoding.RLE);
+ }
+ encodingList.add(vectorEncoding);
+
+ List<CompressionType> compressionTypeList = new ArrayList<>();
+ compressionTypeList.add(CompressionType.SNAPPY);
+
+ session.createDeviceTemplate(
+ "template1", measurementList, dataTypeList, encodingList, compressionTypeList);
+ session.setDeviceTemplate("template1", "root.sg_1");
+ }
+
+ /** Method 1 for insert tablet with aligned timeseries */
+ private static void insertTabletWithAlignedTimeseriesMethod1()
+ throws IoTDBConnectionException, StatementExecutionException {
+ // The schema of measurements of one device
+ // only measurementId and data type in MeasurementSchema take effects in Tablet
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
+ schemaList.add(
+ new VectorMeasurementSchema(
+ new String[] {"s1", "s2"}, new TSDataType[] {TSDataType.INT64, TSDataType.INT32}));
+
+ Tablet tablet = new Tablet(ROOT_SG1_D1, schemaList);
+ long timestamp = System.currentTimeMillis();
+
+ for (long row = 0; row < 100; row++) {
+ int rowIndex = tablet.rowSize++;
+ tablet.addTimestamp(rowIndex, timestamp);
+ tablet.addValue(
+ schemaList.get(0).getValueMeasurementIdList().get(0),
+ rowIndex,
+ new SecureRandom().nextLong());
+ tablet.addValue(
+ schemaList.get(0).getValueMeasurementIdList().get(1),
+ rowIndex,
+ new SecureRandom().nextInt());
+
+ if (tablet.rowSize == tablet.getMaxRowNumber()) {
+ session.insertTablet(tablet, true);
+ tablet.reset();
+ }
+ timestamp++;
+ }
+
+ if (tablet.rowSize != 0) {
+ session.insertTablet(tablet);
+ tablet.reset();
+ }
+
+ session.executeNonQueryStatement("flush");
+ }
+
+ /** Method 2 for insert tablet with aligned timeseries */
+ private static void insertTabletWithAlignedTimeseriesMethod2()
+ throws IoTDBConnectionException, StatementExecutionException {
+ // The schema of measurements of one device
+ // only measurementId and data type in MeasurementSchema take effects in Tablet
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
+ schemaList.add(
+ new VectorMeasurementSchema(
+ new String[] {"s1", "s2"}, new TSDataType[] {TSDataType.INT64, TSDataType.INT32}));
+
+ Tablet tablet = new Tablet(ROOT_SG1_D1, schemaList);
+ long[] timestamps = tablet.timestamps;
+ Object[] values = tablet.values;
+
+ for (long time = 0; time < 100; time++) {
+ int row = tablet.rowSize++;
+ timestamps[row] = time;
+
+ long[] sensor1 = (long[]) values[0];
+ sensor1[row] = new SecureRandom().nextLong();
+
+ int[] sensor2 = (int[]) values[1];
+ sensor2[row] = new SecureRandom().nextInt();
+
+ if (tablet.rowSize == tablet.getMaxRowNumber()) {
+ session.insertTablet(tablet, true);
+ tablet.reset();
+ }
+ }
+
+ if (tablet.rowSize != 0) {
+ session.insertTablet(tablet, true);
+ tablet.reset();
+ }
+
+ session.executeNonQueryStatement("flush");
+ }
+
+ private static void insertNullableTabletWithAlignedTimeseries()
+ throws IoTDBConnectionException, StatementExecutionException {
+ // The schema of measurements of one device
+ // only measurementId and data type in MeasurementSchema take effects in Tablet
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
+ schemaList.add(
+ new VectorMeasurementSchema(
+ new String[] {"s1", "s2"}, new TSDataType[] {TSDataType.INT64, TSDataType.INT32}));
+
+ Tablet tablet = new Tablet(ROOT_SG1_D1, schemaList);
+
+ long[] timestamps = tablet.timestamps;
+ Object[] values = tablet.values;
+ // Use the bitMap to mark the null value point
+ BitMap[] bitMaps = new BitMap[values.length];
+ tablet.bitMaps = bitMaps;
+
+ bitMaps[1] = new BitMap(tablet.getMaxRowNumber());
+ for (long time = 100; time < 200; time++) {
+ int row = tablet.rowSize++;
+ timestamps[row] = time;
+
+ long[] sensor1 = (long[]) values[0];
+ sensor1[row] = new SecureRandom().nextLong();
+
+ int[] sensor2 = (int[]) values[1];
+ sensor2[row] = new SecureRandom().nextInt();
+
+ // mark this point as null value
+ if (time % 5 == 0) {
+ bitMaps[1].mark(row);
+ }
+
+ if (tablet.rowSize == tablet.getMaxRowNumber()) {
+ session.insertTablet(tablet, true);
+ tablet.reset();
+ bitMaps[1].reset();
+ }
+ }
+
+ if (tablet.rowSize != 0) {
+ session.insertTablet(tablet, true);
+ tablet.reset();
+ }
+
+ session.executeNonQueryStatement("flush");
+ }
+}
diff --git a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileWriteWithTablet.java b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileWriteWithTablet.java
index d10094f..541e371 100644
--- a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileWriteWithTablet.java
+++ b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileWriteWithTablet.java
@@ -25,6 +25,7 @@
import org.apache.iotdb.tsfile.read.common.Path;
import org.apache.iotdb.tsfile.write.TsFileWriter;
import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.Schema;
@@ -57,10 +58,10 @@
// the number of values to include in the tablet
int sensorNum = 10;
- List<MeasurementSchema> measurementSchemas = new ArrayList<>();
+ List<IMeasurementSchema> measurementSchemas = new ArrayList<>();
// add measurements into file schema (all with INT64 data type)
for (int i = 0; i < sensorNum; i++) {
- MeasurementSchema measurementSchema =
+ IMeasurementSchema measurementSchema =
new MeasurementSchema(sensorPrefix + (i + 1), TSDataType.INT64, TSEncoding.TS_2DIFF);
measurementSchemas.add(measurementSchema);
schema.registerTimeseries(
diff --git a/hadoop/src/test/java/org/apache/iotdb/hadoop/tsfile/TsFileTestHelper.java b/hadoop/src/test/java/org/apache/iotdb/hadoop/tsfile/TsFileTestHelper.java
index b48b6f6..f04a809 100644
--- a/hadoop/src/test/java/org/apache/iotdb/hadoop/tsfile/TsFileTestHelper.java
+++ b/hadoop/src/test/java/org/apache/iotdb/hadoop/tsfile/TsFileTestHelper.java
@@ -24,6 +24,7 @@
import org.apache.iotdb.tsfile.read.common.Path;
import org.apache.iotdb.tsfile.write.TsFileWriter;
import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.Schema;
@@ -54,7 +55,7 @@
}
Schema schema = new Schema();
- List<MeasurementSchema> schemaList = new ArrayList<>();
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
// the number of rows to include in the tablet
int rowNum = 1000000;
diff --git a/hive-connector/src/test/java/org/apache/iotdb/hive/TsFileTestHelper.java b/hive-connector/src/test/java/org/apache/iotdb/hive/TsFileTestHelper.java
index 649d6ee..4cf8097 100644
--- a/hive-connector/src/test/java/org/apache/iotdb/hive/TsFileTestHelper.java
+++ b/hive-connector/src/test/java/org/apache/iotdb/hive/TsFileTestHelper.java
@@ -25,6 +25,7 @@
import org.apache.iotdb.tsfile.read.common.Path;
import org.apache.iotdb.tsfile.write.TsFileWriter;
import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.Schema;
@@ -55,7 +56,7 @@
}
Schema schema = new Schema();
- List<MeasurementSchema> schemaList = new ArrayList<>();
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
// the number of rows to include in the tablet
int rowNum = 1000000;
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
index a793cfe..f5bba2e 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
@@ -112,6 +112,7 @@
public static final String PATH_WILDCARD = "*";
public static final String TIME = "time";
+ public static final String ALIGN_TIMESERIES_PREFIX = "$#$";
// sdt parameters
public static final String LOSS = "loss";
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
index 3b5d222..861d87d 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
@@ -663,10 +663,9 @@
throws StorageEngineException, QueryProcessException {
PartialPath fullPath = (PartialPath) seriesExpression.getSeriesPath();
PartialPath deviceId = fullPath.getDevicePath();
- String measurementId = seriesExpression.getSeriesPath().getMeasurement();
StorageGroupProcessor storageGroupProcessor = getProcessor(deviceId);
return storageGroupProcessor.query(
- deviceId, measurementId, context, filePathsManager, seriesExpression.getFilter());
+ fullPath, context, filePathsManager, seriesExpression.getFilter());
}
/**
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cache/CacheHitRatioMonitor.java b/server/src/main/java/org/apache/iotdb/db/engine/cache/CacheHitRatioMonitor.java
index 7589f54..51e9b1b 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/cache/CacheHitRatioMonitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/cache/CacheHitRatioMonitor.java
@@ -19,7 +19,10 @@
package org.apache.iotdb.db.engine.cache;
import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.engine.flush.FlushManager;
import org.apache.iotdb.db.exception.StartupException;
+import org.apache.iotdb.db.rescon.MemTableManager;
+import org.apache.iotdb.db.rescon.SystemInfo;
import org.apache.iotdb.db.service.IService;
import org.apache.iotdb.db.service.JMXService;
import org.apache.iotdb.db.service.ServiceType;
@@ -112,4 +115,29 @@
private AsyncCacheHitRatioHolder() {}
}
+
+ @Override
+ public long getTotalMemTableSize() {
+ return SystemInfo.getInstance().getTotalMemTableSize();
+ }
+
+ @Override
+ public double getFlushThershold() {
+ return SystemInfo.getInstance().getFlushThershold();
+ }
+
+ @Override
+ public double getRejectThershold() {
+ return SystemInfo.getInstance().getRejectThershold();
+ }
+
+ @Override
+ public int flushingMemTableNum() {
+ return FlushManager.getInstance().getNumberOfWorkingTasks();
+ }
+
+ @Override
+ public int totalMemTableNum() {
+ return MemTableManager.getInstance().getCurrentMemtableNumber();
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cache/CacheHitRatioMonitorMXBean.java b/server/src/main/java/org/apache/iotdb/db/engine/cache/CacheHitRatioMonitorMXBean.java
index 1c7d261..854fb1a 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/cache/CacheHitRatioMonitorMXBean.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/cache/CacheHitRatioMonitorMXBean.java
@@ -39,4 +39,14 @@
double getTimeSeriesCacheUsedMemoryProportion();
long getTimeSeriesMetaDataCacheAverageSize();
+
+ long getTotalMemTableSize();
+
+ double getFlushThershold();
+
+ double getRejectThershold();
+
+ int flushingMemTableNum();
+
+ int totalMemTableNum();
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cache/TimeSeriesMetadataCache.java b/server/src/main/java/org/apache/iotdb/db/engine/cache/TimeSeriesMetadataCache.java
index 0fe95da..88d076d 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/cache/TimeSeriesMetadataCache.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/cache/TimeSeriesMetadataCache.java
@@ -25,6 +25,7 @@
import org.apache.iotdb.db.query.control.FileReaderManager;
import org.apache.iotdb.db.utils.TestOnly;
import org.apache.iotdb.tsfile.common.cache.Accountable;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import org.apache.iotdb.tsfile.read.common.Path;
@@ -90,7 +91,7 @@
+ RamUsageEstimator.shallowSizeOf(value)
+ RamUsageEstimator.sizeOf(value.getMeasurementId())
+ RamUsageEstimator.shallowSizeOf(value.getStatistics())
- + (value.getChunkMetadataList().get(0).calculateRamSize()
+ + (((ChunkMetadata) value.getChunkMetadataList().get(0)).calculateRamSize()
+ RamUsageEstimator.NUM_BYTES_OBJECT_REF)
* value.getChunkMetadataList().size()
+ RamUsageEstimator.shallowSizeOf(value.getChunkMetadataList());
@@ -106,7 +107,7 @@
+ RamUsageEstimator.shallowSizeOf(value)
+ RamUsageEstimator.sizeOf(value.getMeasurementId())
+ RamUsageEstimator.shallowSizeOf(value.getStatistics())
- + (value.getChunkMetadataList().get(0).calculateRamSize()
+ + (((ChunkMetadata) value.getChunkMetadataList().get(0)).calculateRamSize()
+ RamUsageEstimator.NUM_BYTES_OBJECT_REF)
* value.getChunkMetadataList().size()
+ RamUsageEstimator.shallowSizeOf(value.getChunkMetadataList());
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java b/server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java
index 9397af1..633dd6e 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java
@@ -26,17 +26,20 @@
import org.apache.iotdb.db.exception.runtime.FlushRunTimeException;
import org.apache.iotdb.db.rescon.SystemInfo;
import org.apache.iotdb.db.utils.datastructure.TVList;
+import org.apache.iotdb.db.utils.datastructure.VectorTVList;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.utils.Pair;
import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.chunk.VectorChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
+import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
@@ -101,7 +104,7 @@
long start = System.currentTimeMillis();
long sortTime = 0;
- // for map do not use get(key) to iteratate
+ // for map do not use get(key) to iterate
for (Map.Entry<String, Map<String, IWritableMemChunk>> memTableEntry :
memTable.getMemTableMap().entrySet()) {
encodingTaskQueue.put(new StartFlushGroupIOTask(memTableEntry.getKey()));
@@ -110,8 +113,8 @@
for (Map.Entry<String, IWritableMemChunk> iWritableMemChunkEntry : value.entrySet()) {
long startTime = System.currentTimeMillis();
IWritableMemChunk series = iWritableMemChunkEntry.getValue();
- MeasurementSchema desc = series.getSchema();
- TVList tvList = series.getSortedTVListForFlush();
+ IMeasurementSchema desc = series.getSchema();
+ TVList tvList = series.getSortedTvListForFlush();
sortTime += System.currentTimeMillis() - startTime;
encodingTaskQueue.put(new Pair<>(tvList, desc));
}
@@ -158,37 +161,91 @@
new Runnable() {
private void writeOneSeries(
TVList tvPairs, IChunkWriter seriesWriterImpl, TSDataType dataType) {
- for (int i = 0; i < tvPairs.size(); i++) {
- long time = tvPairs.getTime(i);
+ for (int sortedRowIndex = 0; sortedRowIndex < tvPairs.size(); sortedRowIndex++) {
+ long time = tvPairs.getTime(sortedRowIndex);
// skip duplicated data
- if ((i + 1 < tvPairs.size() && (time == tvPairs.getTime(i + 1)))) {
+ if ((sortedRowIndex + 1 < tvPairs.size()
+ && (time == tvPairs.getTime(sortedRowIndex + 1)))) {
continue;
}
// store last point for SDT
- if (i + 1 == tvPairs.size()) {
+ if (dataType != TSDataType.VECTOR && sortedRowIndex + 1 == tvPairs.size()) {
((ChunkWriterImpl) seriesWriterImpl).setLastPoint(true);
}
switch (dataType) {
case BOOLEAN:
- seriesWriterImpl.write(time, tvPairs.getBoolean(i));
+ seriesWriterImpl.write(time, tvPairs.getBoolean(sortedRowIndex), false);
break;
case INT32:
- seriesWriterImpl.write(time, tvPairs.getInt(i));
+ seriesWriterImpl.write(time, tvPairs.getInt(sortedRowIndex), false);
break;
case INT64:
- seriesWriterImpl.write(time, tvPairs.getLong(i));
+ seriesWriterImpl.write(time, tvPairs.getLong(sortedRowIndex), false);
break;
case FLOAT:
- seriesWriterImpl.write(time, tvPairs.getFloat(i));
+ seriesWriterImpl.write(time, tvPairs.getFloat(sortedRowIndex), false);
break;
case DOUBLE:
- seriesWriterImpl.write(time, tvPairs.getDouble(i));
+ seriesWriterImpl.write(time, tvPairs.getDouble(sortedRowIndex), false);
break;
case TEXT:
- seriesWriterImpl.write(time, tvPairs.getBinary(i));
+ seriesWriterImpl.write(time, tvPairs.getBinary(sortedRowIndex), false);
+ break;
+ case VECTOR:
+ VectorTVList vectorTvPairs = (VectorTVList) tvPairs;
+ List<TSDataType> dataTypes = vectorTvPairs.getTsDataTypes();
+ int originRowIndex = vectorTvPairs.getValueIndex(sortedRowIndex);
+ for (int columnIndex = 0; columnIndex < dataTypes.size(); columnIndex++) {
+ boolean isNull = vectorTvPairs.isValueMarked(originRowIndex, columnIndex);
+ switch (dataTypes.get(columnIndex)) {
+ case BOOLEAN:
+ seriesWriterImpl.write(
+ time,
+ vectorTvPairs.getBooleanByValueIndex(originRowIndex, columnIndex),
+ isNull);
+ break;
+ case INT32:
+ seriesWriterImpl.write(
+ time,
+ vectorTvPairs.getIntByValueIndex(originRowIndex, columnIndex),
+ isNull);
+ break;
+ case INT64:
+ seriesWriterImpl.write(
+ time,
+ vectorTvPairs.getLongByValueIndex(originRowIndex, columnIndex),
+ isNull);
+ break;
+ case FLOAT:
+ seriesWriterImpl.write(
+ time,
+ vectorTvPairs.getFloatByValueIndex(originRowIndex, columnIndex),
+ isNull);
+ break;
+ case DOUBLE:
+ seriesWriterImpl.write(
+ time,
+ vectorTvPairs.getDoubleByValueIndex(originRowIndex, columnIndex),
+ isNull);
+ break;
+ case TEXT:
+ seriesWriterImpl.write(
+ time,
+ vectorTvPairs.getBinaryByValueIndex(originRowIndex, columnIndex),
+ isNull);
+ break;
+ default:
+ LOGGER.error(
+ "Storage group {} does not support data type: {}",
+ storageGroup,
+ dataType);
+ break;
+ }
+ }
+ seriesWriterImpl.write(time);
break;
default:
LOGGER.error(
@@ -233,9 +290,14 @@
break;
} else {
long starTime = System.currentTimeMillis();
- Pair<TVList, MeasurementSchema> encodingMessage =
- (Pair<TVList, MeasurementSchema>) task;
- IChunkWriter seriesWriter = new ChunkWriterImpl(encodingMessage.right);
+ Pair<TVList, IMeasurementSchema> encodingMessage =
+ (Pair<TVList, IMeasurementSchema>) task;
+ IChunkWriter seriesWriter;
+ if (encodingMessage.left.getDataType() == TSDataType.VECTOR) {
+ seriesWriter = new VectorChunkWriterImpl(encodingMessage.right);
+ } else {
+ seriesWriter = new ChunkWriterImpl(encodingMessage.right);
+ }
writeOneSeries(encodingMessage.left, seriesWriter, encodingMessage.right.getType());
seriesWriter.sealCurrentPage();
seriesWriter.clearPageWriter();
@@ -285,9 +347,12 @@
this.writer.startChunkGroup(((StartFlushGroupIOTask) ioMessage).deviceId);
} else if (ioMessage instanceof TaskEnd) {
break;
- } else if (ioMessage instanceof IChunkWriter) {
+ } else if (ioMessage instanceof ChunkWriterImpl) {
ChunkWriterImpl chunkWriter = (ChunkWriterImpl) ioMessage;
chunkWriter.writeToFileWriter(this.writer);
+ } else if (ioMessage instanceof VectorChunkWriterImpl) {
+ VectorChunkWriterImpl chunkWriter = (VectorChunkWriterImpl) ioMessage;
+ chunkWriter.writeToFileWriter(this.writer);
} else {
this.writer.setMinPlanIndex(memTable.getMinPlanIndex());
this.writer.setMaxPlanIndex(memTable.getMaxPlanIndex());
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/flush/NotifyFlushMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/flush/NotifyFlushMemTable.java
index 5ba50d0..4aeb66c 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/flush/NotifyFlushMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/flush/NotifyFlushMemTable.java
@@ -21,7 +21,7 @@
import org.apache.iotdb.db.engine.memtable.AbstractMemTable;
import org.apache.iotdb.db.engine.memtable.IMemTable;
import org.apache.iotdb.db.engine.memtable.IWritableMemChunk;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
/**
* Only used in sync flush and async close to start a flush task This memtable is not managed by
@@ -30,7 +30,7 @@
public class NotifyFlushMemTable extends AbstractMemTable {
@Override
- protected IWritableMemChunk genMemSeries(MeasurementSchema schema) {
+ protected IWritableMemChunk genMemSeries(IMeasurementSchema schema) {
return null;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
index 21c9bf2..57e0e51 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
@@ -21,8 +21,8 @@
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
import org.apache.iotdb.db.exception.WriteProcessException;
-import org.apache.iotdb.db.exception.metadata.MetadataException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.MetaUtils;
import org.apache.iotdb.db.metadata.PartialPath;
import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
@@ -31,11 +31,13 @@
import org.apache.iotdb.db.utils.MemUtils;
import org.apache.iotdb.db.utils.datastructure.TVList;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import org.apache.iotdb.tsfile.read.common.TimeRange;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.utils.BitMap;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
@@ -92,13 +94,12 @@
return memTableMap.containsKey(deviceId) && memTableMap.get(deviceId).containsKey(measurement);
}
- private IWritableMemChunk createIfNotExistAndGet(
- String deviceId, String measurement, MeasurementSchema schema) {
+ private IWritableMemChunk createIfNotExistAndGet(String deviceId, IMeasurementSchema schema) {
Map<String, IWritableMemChunk> memSeries =
memTableMap.computeIfAbsent(deviceId, k -> new HashMap<>());
return memSeries.computeIfAbsent(
- measurement,
+ schema.getMeasurementId(),
k -> {
seriesNumber++;
totalPointsNumThreshold += avgSeriesPointNumThreshold;
@@ -106,7 +107,7 @@
});
}
- protected abstract IWritableMemChunk genMemSeries(MeasurementSchema schema);
+ protected abstract IWritableMemChunk genMemSeries(IMeasurementSchema schema);
@Override
public void insert(InsertRowPlan insertRowPlan) {
@@ -114,23 +115,44 @@
Object[] values = insertRowPlan.getValues();
MeasurementMNode[] measurementMNodes = insertRowPlan.getMeasurementMNodes();
- String[] measurements = insertRowPlan.getMeasurements();
- for (int i = 0; i < values.length; i++) {
- Object value = values[i];
- if (value == null) {
- continue;
+ int columnIndex = 0;
+ for (int i = 0; i < measurementMNodes.length; i++) {
+
+ if (measurementMNodes[i] != null
+ && measurementMNodes[i].getSchema().getType() == TSDataType.VECTOR) {
+ // write vector
+ Object[] vectorValue =
+ new Object[measurementMNodes[i].getSchema().getValueTSDataTypeList().size()];
+ for (int j = 0; j < vectorValue.length; j++) {
+ vectorValue[j] = values[columnIndex];
+ columnIndex++;
+ }
+ memSize +=
+ MemUtils.getVectorRecordSize(
+ measurementMNodes[i].getSchema().getValueTSDataTypeList(),
+ vectorValue,
+ disableMemControl);
+ write(
+ insertRowPlan.getDeviceId().getFullPath(),
+ measurementMNodes[i].getSchema(),
+ insertRowPlan.getTime(),
+ vectorValue);
+ } else {
+ if (values[columnIndex] == null) {
+ columnIndex++;
+ continue;
+ }
+ memSize +=
+ MemUtils.getRecordSize(
+ measurementMNodes[i].getSchema().getType(), values[columnIndex], disableMemControl);
+
+ write(
+ insertRowPlan.getDeviceId().getFullPath(),
+ measurementMNodes[i].getSchema(),
+ insertRowPlan.getTime(),
+ values[columnIndex]);
+ columnIndex++;
}
-
- memSize +=
- MemUtils.getRecordSize(
- measurementMNodes[i].getSchema().getType(), value, disableMemControl);
-
- write(
- insertRowPlan.getDeviceId().getFullPath(),
- measurements[i],
- measurementMNodes[i].getSchema(),
- insertRowPlan.getTime(),
- value);
}
totalPointsNum +=
@@ -145,8 +167,7 @@
write(insertTabletPlan, start, end);
memSize += MemUtils.getRecordSize(insertTabletPlan, start, end, disableMemControl);
totalPointsNum +=
- (insertTabletPlan.getMeasurements().length
- - insertTabletPlan.getFailedMeasurementNumber())
+ (insertTabletPlan.getDataTypes().length - insertTabletPlan.getFailedMeasurementNumber())
* (end - start);
} catch (RuntimeException e) {
throw new WriteProcessException(e);
@@ -155,33 +176,51 @@
@Override
public void write(
- String deviceId,
- String measurement,
- MeasurementSchema schema,
- long insertTime,
- Object objectValue) {
- IWritableMemChunk memSeries = createIfNotExistAndGet(deviceId, measurement, schema);
+ String deviceId, IMeasurementSchema schema, long insertTime, Object objectValue) {
+ IWritableMemChunk memSeries = createIfNotExistAndGet(deviceId, schema);
memSeries.write(insertTime, objectValue);
}
+ @SuppressWarnings("squid:S3776") // high Cognitive Complexity
@Override
public void write(InsertTabletPlan insertTabletPlan, int start, int end) {
+ int columnIndex = 0;
updatePlanIndexes(insertTabletPlan.getIndex());
for (int i = 0; i < insertTabletPlan.getMeasurements().length; i++) {
- if (insertTabletPlan.getColumns()[i] == null) {
+ if (insertTabletPlan.getColumns()[columnIndex] == null) {
+ columnIndex++;
continue;
}
IWritableMemChunk memSeries =
createIfNotExistAndGet(
insertTabletPlan.getDeviceId().getFullPath(),
- insertTabletPlan.getMeasurements()[i],
insertTabletPlan.getMeasurementMNodes()[i].getSchema());
- memSeries.write(
- insertTabletPlan.getTimes(),
- insertTabletPlan.getColumns()[i],
- insertTabletPlan.getDataTypes()[i],
- start,
- end);
+ if (insertTabletPlan.getMeasurementMNodes()[i].getSchema().getType() == TSDataType.VECTOR) {
+ VectorMeasurementSchema vectorSchema =
+ (VectorMeasurementSchema) insertTabletPlan.getMeasurementMNodes()[i].getSchema();
+ Object[] columns = new Object[vectorSchema.getValueMeasurementIdList().size()];
+ BitMap[] bitMaps = new BitMap[vectorSchema.getValueMeasurementIdList().size()];
+ for (int j = 0; j < vectorSchema.getValueMeasurementIdList().size(); j++) {
+ columns[j] = insertTabletPlan.getColumns()[columnIndex];
+ if (insertTabletPlan.getBitMaps() != null) {
+ bitMaps[j] = insertTabletPlan.getBitMaps()[columnIndex];
+ }
+ columnIndex++;
+ }
+ memSeries.write(
+ insertTabletPlan.getTimes(), bitMaps, columns, TSDataType.VECTOR, start, end);
+ } else {
+ memSeries.write(
+ insertTabletPlan.getTimes(),
+ insertTabletPlan.getBitMaps() != null
+ ? insertTabletPlan.getBitMaps()[columnIndex]
+ : null,
+ insertTabletPlan.getColumns()[columnIndex],
+ insertTabletPlan.getDataTypes()[columnIndex],
+ start,
+ end);
+ columnIndex++;
+ }
}
}
@@ -256,24 +295,59 @@
public ReadOnlyMemChunk query(
String deviceId,
String measurement,
- TSDataType dataType,
- TSEncoding encoding,
- Map<String, String> props,
+ IMeasurementSchema partialVectorSchema,
long timeLowerBound,
List<TimeRange> deletionList)
- throws IOException, QueryProcessException, MetadataException {
- if (!checkPath(deviceId, measurement)) {
- return null;
- }
- IWritableMemChunk memChunk = memTableMap.get(deviceId).get(measurement);
- // get sorted tv list is synchronized so different query can get right sorted list reference
- TVList chunkCopy = memChunk.getSortedTVListForQuery();
- int curSize = chunkCopy.size();
+ throws IOException, QueryProcessException {
+ if (partialVectorSchema.getType() == TSDataType.VECTOR) {
+ if (!memTableMap.containsKey(deviceId)) {
+ return null;
+ }
+ IWritableMemChunk vectorMemChunk =
+ memTableMap.get(deviceId).get(partialVectorSchema.getMeasurementId());
- return new ReadOnlyMemChunk(
- measurement, dataType, encoding, chunkCopy, props, curSize, deletionList);
+ List<String> measurementIdList = partialVectorSchema.getValueMeasurementIdList();
+ List<Integer> columns = new ArrayList<>();
+ IMeasurementSchema vectorSchema = vectorMemChunk.getSchema();
+ for (String queryingMeasurement : measurementIdList) {
+ columns.add(vectorSchema.getValueMeasurementIdList().indexOf(queryingMeasurement));
+ }
+ // get sorted tv list is synchronized so different query can get right sorted list reference
+ TVList vectorTvListCopy = vectorMemChunk.getSortedTvListForQuery(columns);
+ int curSize = vectorTvListCopy.size();
+ // return normal ReadOnlyMemChunk for query one measurement in vector
+ if (columns.size() == 1) {
+ return new ReadOnlyMemChunk(
+ measurementIdList.get(0),
+ partialVectorSchema.getValueTSDataTypeList().get(0),
+ partialVectorSchema.getValueTSEncodingList().get(0),
+ vectorTvListCopy,
+ null,
+ curSize,
+ deletionList);
+ }
+ return new ReadOnlyMemChunk(partialVectorSchema, vectorTvListCopy, curSize, deletionList);
+ } else {
+ if (!checkPath(deviceId, measurement)) {
+ return null;
+ }
+ IWritableMemChunk memChunk =
+ memTableMap.get(deviceId).get(partialVectorSchema.getMeasurementId());
+ // get sorted tv list is synchronized so different query can get right sorted list reference
+ TVList chunkCopy = memChunk.getSortedTvListForQuery();
+ int curSize = chunkCopy.size();
+ return new ReadOnlyMemChunk(
+ measurement,
+ partialVectorSchema.getType(),
+ partialVectorSchema.getEncodingType(),
+ chunkCopy,
+ partialVectorSchema.getProps(),
+ curSize,
+ deletionList);
+ }
}
+ @SuppressWarnings("squid:S3776") // high Cognitive Complexity
@Override
public void delete(
PartialPath originalPath, PartialPath devicePath, long startTimestamp, long endTimestamp) {
@@ -287,6 +361,7 @@
Entry<String, IWritableMemChunk> entry = iter.next();
IWritableMemChunk chunk = entry.getValue();
PartialPath fullPath = devicePath.concatNode(entry.getKey());
+ IMeasurementSchema schema = chunk.getSchema();
if (originalPath.matchFullPath(fullPath)) {
if (startTimestamp == Long.MIN_VALUE && endTimestamp == Long.MAX_VALUE) {
iter.remove();
@@ -294,6 +369,17 @@
int deletedPointsNumber = chunk.delete(startTimestamp, endTimestamp);
totalPointsNum -= deletedPointsNumber;
}
+ // for vector type
+ else if (schema.getType() == TSDataType.VECTOR) {
+ List<String> measurements = MetaUtils.getMeasurementsInPartialPath(originalPath);
+ if (measurements.containsAll(schema.getValueMeasurementIdList())) {
+ if (startTimestamp == Long.MIN_VALUE && endTimestamp == Long.MAX_VALUE) {
+ iter.remove();
+ }
+ int deletedPointsNumber = chunk.delete(startTimestamp, endTimestamp);
+ totalPointsNum -= deletedPointsNumber;
+ }
+ }
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
index ce412a2..8724e83 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
@@ -25,10 +25,8 @@
import org.apache.iotdb.db.metadata.PartialPath;
import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import org.apache.iotdb.tsfile.read.common.TimeRange;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import java.io.IOException;
import java.util.List;
@@ -46,12 +44,7 @@
Map<String, Map<String, IWritableMemChunk>> getMemTableMap();
- void write(
- String deviceId,
- String measurement,
- MeasurementSchema schema,
- long insertTime,
- Object objectValue);
+ void write(String deviceId, IMeasurementSchema schema, long insertTime, Object objectValue);
void write(InsertTabletPlan insertTabletPlan, int start, int end);
@@ -87,9 +80,7 @@
ReadOnlyMemChunk query(
String deviceId,
String measurement,
- TSDataType dataType,
- TSEncoding encoding,
- Map<String, String> props,
+ IMeasurementSchema schema,
long timeLowerBound,
List<TimeRange> deletionList)
throws IOException, QueryProcessException, MetadataException;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunk.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunk.java
index bdb4bbc..6cb896f 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunk.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunk.java
@@ -21,7 +21,10 @@
import org.apache.iotdb.db.utils.datastructure.TVList;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.utils.Binary;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.utils.BitMap;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.List;
public interface IWritableMemChunk {
@@ -37,26 +40,31 @@
void putBoolean(long t, boolean v);
- void putLongs(long[] t, long[] v, int start, int end);
+ void putVector(long t, Object[] v);
- void putInts(long[] t, int[] v, int start, int end);
+ void putLongs(long[] t, BitMap bitMap, long[] v, int start, int end);
- void putFloats(long[] t, float[] v, int start, int end);
+ void putInts(long[] t, BitMap bitMap, int[] v, int start, int end);
- void putDoubles(long[] t, double[] v, int start, int end);
+ void putFloats(long[] t, BitMap bitMap, float[] v, int start, int end);
- void putBinaries(long[] t, Binary[] v, int start, int end);
+ void putDoubles(long[] t, BitMap bitMap, double[] v, int start, int end);
- void putBooleans(long[] t, boolean[] v, int start, int end);
+ void putBinaries(long[] t, BitMap bitMap, Binary[] v, int start, int end);
+
+ void putBooleans(long[] t, BitMap bitMap, boolean[] v, int start, int end);
+
+ void putVectors(long[] t, BitMap[] bitMaps, Object[] v, int start, int end);
void write(long insertTime, Object objectValue);
/** [start, end) */
- void write(long[] times, Object valueList, TSDataType dataType, int start, int end);
+ void write(
+ long[] times, Object bitMaps, Object valueList, TSDataType dataType, int start, int end);
long count();
- MeasurementSchema getSchema();
+ IMeasurementSchema getSchema();
/**
* served for query requests.
@@ -72,7 +80,17 @@
*
* @return sorted tv list
*/
- TVList getSortedTVListForQuery();
+ TVList getSortedTvListForQuery();
+
+ /**
+ * served for vector query requests.
+ *
+ * <p>the mechanism is just like copy on write
+ *
+ * @param columnIndexList indices of queried columns in the full VectorTVList
+ * @return sorted tv list
+ */
+ TVList getSortedTvListForQuery(List<Integer> columnIndexList);
/**
* served for flush requests. The logic is just same as getSortedTVListForQuery, but without add
@@ -80,7 +98,7 @@
*
* @return sorted tv list
*/
- TVList getSortedTVListForFlush();
+ TVList getSortedTvListForFlush();
default TVList getTVList() {
return null;
@@ -92,4 +110,7 @@
/** @return how many points are deleted */
int delete(long lowerBound, long upperBound);
+
+ // For delete one column in the vector
+ int delete(long lowerBound, long upperBound, int columnIndex);
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java
index 254d722..e13bbf2 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java
@@ -20,7 +20,8 @@
package org.apache.iotdb.db.engine.memtable;
import org.apache.iotdb.db.rescon.TVListAllocator;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import java.util.HashMap;
import java.util.Map;
@@ -38,7 +39,11 @@
}
@Override
- protected IWritableMemChunk genMemSeries(MeasurementSchema schema) {
+ protected IWritableMemChunk genMemSeries(IMeasurementSchema schema) {
+ if (schema.getType() == TSDataType.VECTOR) {
+ return new WritableMemChunk(
+ schema, TVListAllocator.getInstance().allocate(schema.getValueTSDataTypeList()));
+ }
return new WritableMemChunk(schema, TVListAllocator.getInstance().allocate(schema.getType()));
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java
index 4536896..3679cc3 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java
@@ -22,14 +22,18 @@
import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.utils.Binary;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.utils.BitMap;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+
+import java.util.List;
public class WritableMemChunk implements IWritableMemChunk {
- private MeasurementSchema schema;
+ private IMeasurementSchema schema;
private TVList list;
+ private static final String UNSUPPORTED_TYPE = "Unsupported data type:";
- public WritableMemChunk(MeasurementSchema schema, TVList list) {
+ public WritableMemChunk(IMeasurementSchema schema, TVList list) {
this.schema = schema;
this.list = list;
}
@@ -55,40 +59,48 @@
case TEXT:
putBinary(insertTime, (Binary) objectValue);
break;
+ case VECTOR:
+ putVector(insertTime, (Object[]) objectValue);
+ break;
default:
- throw new UnSupportedDataTypeException("Unsupported data type:" + schema.getType());
+ throw new UnSupportedDataTypeException(UNSUPPORTED_TYPE + schema.getType());
}
}
@Override
- public void write(long[] times, Object valueList, TSDataType dataType, int start, int end) {
+ public void write(
+ long[] times, Object bitMap, Object valueList, TSDataType dataType, int start, int end) {
switch (dataType) {
case BOOLEAN:
boolean[] boolValues = (boolean[]) valueList;
- putBooleans(times, boolValues, start, end);
+ putBooleans(times, (BitMap) bitMap, boolValues, start, end);
break;
case INT32:
int[] intValues = (int[]) valueList;
- putInts(times, intValues, start, end);
+ putInts(times, (BitMap) bitMap, intValues, start, end);
break;
case INT64:
long[] longValues = (long[]) valueList;
- putLongs(times, longValues, start, end);
+ putLongs(times, (BitMap) bitMap, longValues, start, end);
break;
case FLOAT:
float[] floatValues = (float[]) valueList;
- putFloats(times, floatValues, start, end);
+ putFloats(times, (BitMap) bitMap, floatValues, start, end);
break;
case DOUBLE:
double[] doubleValues = (double[]) valueList;
- putDoubles(times, doubleValues, start, end);
+ putDoubles(times, (BitMap) bitMap, doubleValues, start, end);
break;
case TEXT:
Binary[] binaryValues = (Binary[]) valueList;
- putBinaries(times, binaryValues, start, end);
+ putBinaries(times, (BitMap) bitMap, binaryValues, start, end);
+ break;
+ case VECTOR:
+ Object[] vectorValues = (Object[]) valueList;
+ putVectors(times, (BitMap[]) bitMap, vectorValues, start, end);
break;
default:
- throw new UnSupportedDataTypeException("Unsupported data type:" + dataType);
+ throw new UnSupportedDataTypeException(UNSUPPORTED_TYPE + dataType);
}
}
@@ -123,43 +135,64 @@
}
@Override
- public void putLongs(long[] t, long[] v, int start, int end) {
+ public void putVector(long t, Object[] v) {
+ list.putVector(t, v);
+ }
+
+ @Override
+ public void putLongs(long[] t, BitMap bitMap, long[] v, int start, int end) {
list.putLongs(t, v, start, end);
}
@Override
- public void putInts(long[] t, int[] v, int start, int end) {
+ public void putInts(long[] t, BitMap bitMap, int[] v, int start, int end) {
list.putInts(t, v, start, end);
}
@Override
- public void putFloats(long[] t, float[] v, int start, int end) {
+ public void putFloats(long[] t, BitMap bitMap, float[] v, int start, int end) {
list.putFloats(t, v, start, end);
}
@Override
- public void putDoubles(long[] t, double[] v, int start, int end) {
+ public void putDoubles(long[] t, BitMap bitMap, double[] v, int start, int end) {
list.putDoubles(t, v, start, end);
}
@Override
- public void putBinaries(long[] t, Binary[] v, int start, int end) {
+ public void putBinaries(long[] t, BitMap bitMap, Binary[] v, int start, int end) {
list.putBinaries(t, v, start, end);
}
@Override
- public void putBooleans(long[] t, boolean[] v, int start, int end) {
+ public void putBooleans(long[] t, BitMap bitMap, boolean[] v, int start, int end) {
list.putBooleans(t, v, start, end);
}
@Override
- public synchronized TVList getSortedTVListForQuery() {
+ public void putVectors(long[] t, BitMap[] bitMaps, Object[] v, int start, int end) {
+ list.putVectors(t, bitMaps, v, start, end);
+ }
+
+ @Override
+ public synchronized TVList getSortedTvListForQuery() {
sortTVList();
// increase reference count
list.increaseReferenceCount();
return list;
}
+ @Override
+ public synchronized TVList getSortedTvListForQuery(List<Integer> columnIndexList) {
+ if (list.getDataType() != TSDataType.VECTOR) {
+ throw new UnSupportedDataTypeException(UNSUPPORTED_TYPE + list.getDataType());
+ }
+ sortTVList();
+ // increase reference count
+ list.increaseReferenceCount();
+ return list.getTvListByColumnIndex(columnIndexList);
+ }
+
private void sortTVList() {
// check reference count
if ((list.getReferenceCount() > 0 && !list.isSorted())) {
@@ -172,7 +205,7 @@
}
@Override
- public synchronized TVList getSortedTVListForFlush() {
+ public synchronized TVList getSortedTvListForFlush() {
sortTVList();
return list;
}
@@ -188,7 +221,7 @@
}
@Override
- public MeasurementSchema getSchema() {
+ public IMeasurementSchema getSchema() {
return schema;
}
@@ -202,17 +235,23 @@
return list.delete(lowerBound, upperBound);
}
+ // TODO: THIS METHOLD IS FOR DELETING ONE COLUMN OF A VECTOR
+ @Override
+ public int delete(long lowerBound, long upperBound, int columnIndex) {
+ return list.delete(lowerBound, upperBound, columnIndex);
+ }
+
@Override
public String toString() {
- int size = getSortedTVListForQuery().size();
+ int size = getSortedTvListForQuery().size();
StringBuilder out = new StringBuilder("MemChunk Size: " + size + System.lineSeparator());
if (size != 0) {
out.append("Data type:").append(schema.getType()).append(System.lineSeparator());
out.append("First point:")
- .append(getSortedTVListForQuery().getTimeValuePair(0))
+ .append(getSortedTvListForQuery().getTimeValuePair(0))
.append(System.lineSeparator());
out.append("Last point:")
- .append(getSortedTVListForQuery().getTimeValuePair(size - 1))
+ .append(getSortedTvListForQuery().getTimeValuePair(size - 1))
.append(System.lineSeparator());
}
return out.toString();
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeResource.java b/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeResource.java
index b91ac86..4e6f6fb 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeResource.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeResource.java
@@ -33,7 +33,7 @@
import org.apache.iotdb.tsfile.utils.Pair;
import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
import java.io.IOException;
@@ -64,9 +64,9 @@
private Map<TsFileResource, List<Modification>> modificationCache = new HashMap<>();
private Map<TsFileResource, Map<String, Pair<Long, Long>>> startEndTimeCache =
new HashMap<>(); // pair<startTime, endTime>
- private Map<PartialPath, MeasurementSchema> measurementSchemaMap =
+ private Map<PartialPath, IMeasurementSchema> measurementSchemaMap =
new HashMap<>(); // is this too waste?
- private Map<MeasurementSchema, IChunkWriter> chunkWriterCache = new ConcurrentHashMap<>();
+ private Map<IMeasurementSchema, IChunkWriter> chunkWriterCache = new ConcurrentHashMap<>();
private long timeLowerBound = Long.MIN_VALUE;
@@ -106,7 +106,7 @@
chunkWriterCache.clear();
}
- public MeasurementSchema getSchema(PartialPath path) {
+ public IMeasurementSchema getSchema(PartialPath path) {
return measurementSchemaMap.get(path);
}
@@ -174,7 +174,7 @@
* Construct the a new or get an existing ChunkWriter of a measurement. Different timeseries of
* the same measurement and data type shares the same instance.
*/
- public IChunkWriter getChunkWriter(MeasurementSchema measurementSchema) {
+ public IChunkWriter getChunkWriter(IMeasurementSchema measurementSchema) {
return chunkWriterCache.computeIfAbsent(measurementSchema, ChunkWriterImpl::new);
}
@@ -259,7 +259,7 @@
this.cacheDeviceMeta = cacheDeviceMeta;
}
- public void setMeasurementSchemaMap(Map<PartialPath, MeasurementSchema> measurementSchemaMap) {
+ public void setMeasurementSchemaMap(Map<PartialPath, IMeasurementSchema> measurementSchemaMap) {
this.measurementSchemaMap = measurementSchemaMap;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeFileTask.java b/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeFileTask.java
index 15b5176..d044bc1 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeFileTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeFileTask.java
@@ -30,6 +30,7 @@
import org.apache.iotdb.db.query.control.FileReaderManager;
import org.apache.iotdb.tsfile.exception.write.TsFileNotCompleteException;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
@@ -227,7 +228,7 @@
for (Entry<String, List<ChunkMetadata>> deviceChunkMetadataListEntry :
deviceChunkMetadataListMap.entrySet()) {
String device = deviceChunkMetadataListEntry.getKey();
- for (ChunkMetadata chunkMetadata : deviceChunkMetadataListEntry.getValue()) {
+ for (IChunkMetadata chunkMetadata : deviceChunkMetadataListEntry.getValue()) {
resource.updateStartTime(seqFile, device, chunkMetadata.getStartTime());
resource.updateEndTime(seqFile, device, chunkMetadata.getEndTime());
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeMultiChunkTask.java b/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeMultiChunkTask.java
index 2b4bb4e..28dad8a 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeMultiChunkTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeMultiChunkTask.java
@@ -40,7 +40,7 @@
import org.apache.iotdb.tsfile.read.reader.IPointReader;
import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
@@ -301,7 +301,7 @@
RestorableTsFileIOWriter mergeFileWriter = resource.getMergeFileWriter(currTsFile);
for (PartialPath path : currMergingPaths) {
- MeasurementSchema schema = resource.getSchema(path);
+ IMeasurementSchema schema = resource.getSchema(path);
mergeFileWriter.addSchema(path, schema);
}
// merge unseq data with seq data in this file or small chunks in this file into a larger chunk
@@ -622,7 +622,7 @@
while (!chunkIdxHeap.isEmpty()) {
int pathIdx = chunkIdxHeap.poll();
PartialPath path = currMergingPaths.get(pathIdx);
- MeasurementSchema measurementSchema = resource.getSchema(path);
+ IMeasurementSchema measurementSchema = resource.getSchema(path);
IChunkWriter chunkWriter = resource.getChunkWriter(measurementSchema);
if (Thread.interrupted()) {
Thread.currentThread().interrupt();
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeTask.java b/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeTask.java
index ec96335..381651c 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeTask.java
@@ -29,7 +29,7 @@
import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
import org.apache.iotdb.db.service.IoTDB;
import org.apache.iotdb.db.utils.MergeUtils;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -146,10 +146,11 @@
mergeLogger.logFiles(resource);
Set<PartialPath> devices = IoTDB.metaManager.getDevices(new PartialPath(storageGroupName));
- Map<PartialPath, MeasurementSchema> measurementSchemaMap = new HashMap<>();
+ Map<PartialPath, IMeasurementSchema> measurementSchemaMap = new HashMap<>();
List<PartialPath> unmergedSeries = new ArrayList<>();
for (PartialPath device : devices) {
MNode deviceNode = IoTDB.metaManager.getNodeByPath(device);
+ // todo add template merge logic
for (Entry<String, MNode> entry : deviceNode.getChildren().entrySet()) {
PartialPath path = device.concatNode(entry.getKey());
measurementSchemaMap.put(path, ((MeasurementMNode) entry.getValue()).getSchema());
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/querycontext/QueryDataSource.java b/server/src/main/java/org/apache/iotdb/db/engine/querycontext/QueryDataSource.java
index b3dfb98..60a6de5 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/querycontext/QueryDataSource.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/querycontext/QueryDataSource.java
@@ -20,7 +20,6 @@
package org.apache.iotdb.db.engine.querycontext;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.metadata.PartialPath;
import org.apache.iotdb.tsfile.read.filter.TimeFilter;
import org.apache.iotdb.tsfile.read.filter.basic.Filter;
import org.apache.iotdb.tsfile.read.filter.operator.AndFilter;
@@ -28,26 +27,17 @@
import java.util.List;
public class QueryDataSource {
- private PartialPath seriesPath;
private List<TsFileResource> seqResources;
private List<TsFileResource> unseqResources;
/** data older than currentTime - dataTTL should be ignored. */
private long dataTTL = Long.MAX_VALUE;
- public QueryDataSource(
- PartialPath seriesPath,
- List<TsFileResource> seqResources,
- List<TsFileResource> unseqResources) {
- this.seriesPath = seriesPath;
+ public QueryDataSource(List<TsFileResource> seqResources, List<TsFileResource> unseqResources) {
this.seqResources = seqResources;
this.unseqResources = unseqResources;
}
- public PartialPath getSeriesPath() {
- return seriesPath;
- }
-
public List<TsFileResource> getSeqResources() {
return seqResources;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java b/server/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java
index 8236e15..883fc00 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java
@@ -24,17 +24,21 @@
import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.VectorChunkMetadata;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
import org.apache.iotdb.tsfile.read.TimeValuePair;
import org.apache.iotdb.tsfile.read.common.TimeRange;
import org.apache.iotdb.tsfile.read.reader.IPointReader;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.List;
import java.util.Map;
@@ -51,7 +55,7 @@
private int floatPrecision = TSFileDescriptor.getInstance().getConfig().getFloatPrecision();
- private ChunkMetadata cachedMetaData;
+ private IChunkMetadata cachedMetaData;
private TVList chunkData;
@@ -99,7 +103,7 @@
private void initChunkMeta() throws IOException, QueryProcessException {
Statistics statsByType = Statistics.getStatsByType(dataType);
- ChunkMetadata metaData = new ChunkMetadata(measurementUid, dataType, 0, statsByType);
+ IChunkMetadata metaData = new ChunkMetadata(measurementUid, dataType, 0, statsByType);
if (!isEmpty()) {
IPointReader iterator =
chunkData.getIterator(floatPrecision, encoding, chunkDataSize, deletionList);
@@ -135,6 +139,102 @@
cachedMetaData = metaData;
}
+ /**
+ * The constructor for VECTOR type.
+ *
+ * @param schema VectorMeasurementSchema
+ * @param tvList VectorTvList
+ * @param size The Number of Chunk data points
+ * @param deletionList The timeRange of deletionList
+ */
+ public ReadOnlyMemChunk(
+ IMeasurementSchema schema, TVList tvList, int size, List<TimeRange> deletionList)
+ throws IOException, QueryProcessException {
+ this.measurementUid = schema.getMeasurementId();
+ this.dataType = schema.getType();
+
+ this.chunkData = tvList;
+ this.chunkDataSize = size;
+ this.deletionList = deletionList;
+
+ this.chunkPointReader =
+ tvList.getIterator(floatPrecision, encoding, chunkDataSize, deletionList);
+ initVectorChunkMeta(schema);
+ }
+
+ @SuppressWarnings("squid:S3776") // high Cognitive Complexity
+ private void initVectorChunkMeta(IMeasurementSchema schema)
+ throws IOException, QueryProcessException {
+ Statistics timeStatistics = Statistics.getStatsByType(TSDataType.VECTOR);
+ IChunkMetadata timeChunkMetadata =
+ new ChunkMetadata(measurementUid, TSDataType.VECTOR, 0, timeStatistics);
+ List<IChunkMetadata> valueChunkMetadataList = new ArrayList<>();
+ Statistics[] valueStatistics = new Statistics[schema.getValueTSDataTypeList().size()];
+ for (int i = 0; i < schema.getValueTSDataTypeList().size(); i++) {
+ valueStatistics[i] = Statistics.getStatsByType(schema.getValueTSDataTypeList().get(i));
+ IChunkMetadata valueChunkMetadata =
+ new ChunkMetadata(
+ schema.getValueMeasurementIdList().get(i),
+ schema.getValueTSDataTypeList().get(i),
+ 0,
+ valueStatistics[i]);
+ valueChunkMetadataList.add(valueChunkMetadata);
+ }
+ if (!isEmpty()) {
+ IPointReader iterator =
+ chunkData.getIterator(floatPrecision, encoding, chunkDataSize, deletionList);
+ while (iterator.hasNextTimeValuePair()) {
+ TimeValuePair timeValuePair = iterator.nextTimeValuePair();
+ timeStatistics.update(timeValuePair.getTimestamp());
+ for (int i = 0; i < schema.getValueTSDataTypeList().size(); i++) {
+ if (timeValuePair.getValue().getVector()[i] == null) {
+ continue;
+ }
+ switch (schema.getValueTSDataTypeList().get(i)) {
+ case BOOLEAN:
+ valueStatistics[i].update(
+ timeValuePair.getTimestamp(),
+ timeValuePair.getValue().getVector()[i].getBoolean());
+ break;
+ case TEXT:
+ valueStatistics[i].update(
+ timeValuePair.getTimestamp(),
+ timeValuePair.getValue().getVector()[i].getBinary());
+ break;
+ case FLOAT:
+ valueStatistics[i].update(
+ timeValuePair.getTimestamp(), timeValuePair.getValue().getVector()[i].getFloat());
+ break;
+ case INT32:
+ valueStatistics[i].update(
+ timeValuePair.getTimestamp(), timeValuePair.getValue().getVector()[i].getInt());
+ break;
+ case INT64:
+ valueStatistics[i].update(
+ timeValuePair.getTimestamp(), timeValuePair.getValue().getVector()[i].getLong());
+ break;
+ case DOUBLE:
+ valueStatistics[i].update(
+ timeValuePair.getTimestamp(),
+ timeValuePair.getValue().getVector()[i].getDouble());
+ break;
+ default:
+ throw new QueryProcessException("Unsupported data type:" + dataType);
+ }
+ }
+ }
+ }
+ timeStatistics.setEmpty(isEmpty());
+ for (Statistics valueStatistic : valueStatistics) {
+ valueStatistic.setEmpty(isEmpty());
+ }
+ IChunkMetadata vectorChunkMetadata =
+ new VectorChunkMetadata(timeChunkMetadata, valueChunkMetadataList);
+ vectorChunkMetadata.setChunkLoader(new MemChunkLoader(this));
+ vectorChunkMetadata.setVersion(Long.MAX_VALUE);
+ cachedMetaData = vectorChunkMetadata;
+ }
+
public TSDataType getDataType() {
return dataType;
}
@@ -143,7 +243,7 @@
return !chunkPointReader.hasNextTimeValuePair();
}
- public ChunkMetadata getChunkMetaData() {
+ public IChunkMetadata getChunkMetaData() {
return cachedMetaData;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
index d828ef8..5dfead5 100755
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
@@ -78,7 +78,7 @@
import org.apache.iotdb.tsfile.read.TimeValuePair;
import org.apache.iotdb.tsfile.read.filter.basic.Filter;
import org.apache.iotdb.tsfile.utils.Pair;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
import org.apache.commons.io.FileUtils;
@@ -1552,8 +1552,7 @@
// TODO need a read lock, please consider the concurrency with flush manager threads.
public QueryDataSource query(
- PartialPath deviceId,
- String measurementId,
+ PartialPath fullPath,
QueryContext context,
QueryFileManager filePathsManager,
Filter timeFilter)
@@ -1564,8 +1563,7 @@
getFileResourceListForQuery(
tsFileManagement.getTsFileList(true),
upgradeSeqFileList,
- deviceId,
- measurementId,
+ fullPath,
context,
timeFilter,
true);
@@ -1573,12 +1571,11 @@
getFileResourceListForQuery(
tsFileManagement.getTsFileList(false),
upgradeUnseqFileList,
- deviceId,
- measurementId,
+ fullPath,
context,
timeFilter,
false);
- QueryDataSource dataSource = new QueryDataSource(deviceId, seqResources, unseqResources);
+ QueryDataSource dataSource = new QueryDataSource(seqResources, unseqResources);
// used files should be added before mergeLock is unlocked, or they may be deleted by
// running merge
// is null only in tests
@@ -1609,24 +1606,24 @@
private List<TsFileResource> getFileResourceListForQuery(
Collection<TsFileResource> tsFileResources,
List<TsFileResource> upgradeTsFileResources,
- PartialPath deviceId,
- String measurementId,
+ PartialPath fullPath,
QueryContext context,
Filter timeFilter,
boolean isSeq)
throws MetadataException {
+ String deviceId = fullPath.getDevice();
if (context.isDebug()) {
DEBUG_LOGGER.info(
"Path: {}.{}, get tsfile list: {} isSeq: {} timefilter: {}",
- deviceId.getFullPath(),
- measurementId,
+ deviceId,
+ fullPath.getMeasurement(),
tsFileResources,
isSeq,
(timeFilter == null ? "null" : timeFilter));
}
- MeasurementSchema schema = IoTDB.metaManager.getSeriesSchema(deviceId, measurementId);
+ IMeasurementSchema schema = IoTDB.metaManager.getSeriesSchema(fullPath);
List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
long timeLowerBound =
@@ -1635,8 +1632,7 @@
// for upgrade files and old files must be closed
for (TsFileResource tsFileResource : upgradeTsFileResources) {
- if (!tsFileResource.isSatisfied(
- deviceId.getFullPath(), timeFilter, isSeq, dataTTL, context.isDebug())) {
+ if (!tsFileResource.isSatisfied(deviceId, timeFilter, isSeq, dataTTL, context.isDebug())) {
continue;
}
closeQueryLock.readLock().lock();
@@ -1649,7 +1645,7 @@
for (TsFileResource tsFileResource : tsFileResources) {
if (!tsFileResource.isSatisfied(
- deviceId.getFullPath(), timeFilter, isSeq, dataTTL, context.isDebug())) {
+ fullPath.getDevice(), timeFilter, isSeq, dataTTL, context.isDebug())) {
continue;
}
closeQueryLock.readLock().lock();
@@ -1659,14 +1655,7 @@
} else {
tsFileResource
.getUnsealedFileProcessor()
- .query(
- deviceId.getFullPath(),
- measurementId,
- schema.getType(),
- schema.getEncodingType(),
- schema.getProps(),
- context,
- tsfileResourcesForQuery);
+ .query(deviceId, fullPath.getMeasurement(), schema, context, tsfileResourcesForQuery);
}
} catch (IOException e) {
throw new MetadataException(e);
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
index c20ec7e..0207971 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
@@ -57,11 +57,14 @@
import org.apache.iotdb.rpc.TSStatusCode;
import org.apache.iotdb.service.rpc.thrift.TSStatus;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.VectorChunkMetadata;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import org.apache.iotdb.tsfile.read.common.TimeRange;
import org.apache.iotdb.tsfile.utils.Binary;
import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
import org.slf4j.Logger;
@@ -73,7 +76,6 @@
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
-import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.locks.ReadWriteLock;
@@ -275,6 +277,7 @@
tsFileResource.updatePlanIndexes(insertTabletPlan.getIndex());
}
+ @SuppressWarnings("squid:S3776") // high Cognitive Complexity
private void checkMemCostAndAddToTspInfo(InsertRowPlan insertRowPlan)
throws WriteProcessException {
// memory of increased PrimitiveArray and TEXT values, e.g., add a long[128], add 128*8
@@ -283,29 +286,43 @@
long chunkMetadataIncrement = 0L;
String deviceId = insertRowPlan.getDeviceId().getFullPath();
long unsealedResourceIncrement = tsFileResource.estimateRamIncrement(deviceId);
- for (int i = 0; i < insertRowPlan.getDataTypes().length; i++) {
+ int columnIndex = 0;
+ for (int i = 0; i < insertRowPlan.getMeasurementMNodes().length; i++) {
// skip failed Measurements
- if (insertRowPlan.getDataTypes()[i] == null || insertRowPlan.getMeasurements()[i] == null) {
+ if (insertRowPlan.getDataTypes()[columnIndex] == null
+ || insertRowPlan.getMeasurements()[i] == null) {
+ columnIndex++;
continue;
}
if (workMemTable.checkIfChunkDoesNotExist(deviceId, insertRowPlan.getMeasurements()[i])) {
// ChunkMetadataIncrement
- chunkMetadataIncrement +=
- ChunkMetadata.calculateRamSize(
- insertRowPlan.getMeasurements()[i], insertRowPlan.getDataTypes()[i]);
- memTableIncrement += TVList.tvListArrayMemSize(insertRowPlan.getDataTypes()[i]);
+ IMeasurementSchema schema = insertRowPlan.getMeasurementMNodes()[i].getSchema();
+ if (schema.getType() == TSDataType.VECTOR) {
+ chunkMetadataIncrement +=
+ schema.getValueTSDataTypeList().size()
+ * ChunkMetadata.calculateRamSize(
+ schema.getValueMeasurementIdList().get(0),
+ schema.getValueTSDataTypeList().get(0));
+ memTableIncrement += TVList.vectorTvListArrayMemSize(schema.getValueTSDataTypeList());
+ } else {
+ chunkMetadataIncrement +=
+ ChunkMetadata.calculateRamSize(
+ insertRowPlan.getMeasurements()[i], insertRowPlan.getDataTypes()[columnIndex]);
+ memTableIncrement += TVList.tvListArrayMemSize(insertRowPlan.getDataTypes()[columnIndex]);
+ }
} else {
// here currentChunkPointNum >= 1
int currentChunkPointNum =
workMemTable.getCurrentChunkPointNum(deviceId, insertRowPlan.getMeasurements()[i]);
memTableIncrement +=
(currentChunkPointNum % PrimitiveArrayManager.ARRAY_SIZE) == 0
- ? TVList.tvListArrayMemSize(insertRowPlan.getDataTypes()[i])
+ ? TVList.tvListArrayMemSize(insertRowPlan.getDataTypes()[columnIndex])
: 0;
}
// TEXT data mem size
- if (insertRowPlan.getDataTypes()[i] == TSDataType.TEXT) {
- textDataIncrement += MemUtils.getBinarySize((Binary) insertRowPlan.getValues()[i]);
+ if (insertRowPlan.getDataTypes()[columnIndex] == TSDataType.TEXT) {
+ textDataIncrement +=
+ MemUtils.getBinarySize((Binary) insertRowPlan.getValues()[columnIndex]);
}
}
updateMemoryInfo(
@@ -322,15 +339,30 @@
String deviceId = insertTabletPlan.getDeviceId().getFullPath();
long unsealedResourceIncrement = tsFileResource.estimateRamIncrement(deviceId);
- for (int i = 0; i < insertTabletPlan.getDataTypes().length; i++) {
- // skip failed Measurements
- TSDataType dataType = insertTabletPlan.getDataTypes()[i];
- String measurement = insertTabletPlan.getMeasurements()[i];
- Object column = insertTabletPlan.getColumns()[i];
- if (dataType == null || column == null || measurement == null) {
- continue;
+ int columnIndex = 0;
+ for (int i = 0; i < insertTabletPlan.getMeasurementMNodes().length; i++) {
+ // for aligned timeseries
+ if (insertTabletPlan.getMeasurementMNodes()[i].getSchema().getType() == TSDataType.VECTOR) {
+ VectorMeasurementSchema vectorSchema =
+ (VectorMeasurementSchema) insertTabletPlan.getMeasurementMNodes()[i].getSchema();
+ Object[] columns = new Object[vectorSchema.getValueMeasurementIdList().size()];
+ for (int j = 0; j < vectorSchema.getValueMeasurementIdList().size(); j++) {
+ columns[j] = insertTabletPlan.getColumns()[columnIndex++];
+ }
+ updateVectorMemCost(vectorSchema, deviceId, start, end, memIncrements, columns);
}
- updateMemCost(dataType, measurement, deviceId, start, end, memIncrements, column);
+ // for non aligned
+ else {
+ // skip failed Measurements
+ TSDataType dataType = insertTabletPlan.getDataTypes()[columnIndex];
+ String measurement = insertTabletPlan.getMeasurements()[i];
+ Object column = insertTabletPlan.getColumns()[columnIndex];
+ columnIndex++;
+ if (dataType == null || column == null || measurement == null) {
+ continue;
+ }
+ updateMemCost(dataType, measurement, deviceId, start, end, memIncrements, column);
+ }
}
long memTableIncrement = memIncrements[0];
long textDataIncrement = memIncrements[1];
@@ -376,6 +408,49 @@
}
}
+ private void updateVectorMemCost(
+ VectorMeasurementSchema vectorSchema,
+ String deviceId,
+ int start,
+ int end,
+ long[] memIncrements,
+ Object[] columns) {
+ // memIncrements = [memTable, text, chunk metadata] respectively
+
+ List<String> measurementIds = vectorSchema.getValueMeasurementIdList();
+ List<TSDataType> dataTypes = vectorSchema.getValueTSDataTypeList();
+ if (workMemTable.checkIfChunkDoesNotExist(deviceId, vectorSchema.getMeasurementId())) {
+ // ChunkMetadataIncrement
+ memIncrements[2] +=
+ dataTypes.size()
+ * ChunkMetadata.calculateRamSize(measurementIds.get(0), dataTypes.get(0));
+ memIncrements[0] +=
+ ((end - start) / PrimitiveArrayManager.ARRAY_SIZE + 1)
+ * TVList.vectorTvListArrayMemSize(dataTypes);
+ } else {
+ int currentChunkPointNum =
+ workMemTable.getCurrentChunkPointNum(deviceId, vectorSchema.getMeasurementId());
+ if (currentChunkPointNum % PrimitiveArrayManager.ARRAY_SIZE == 0) {
+ memIncrements[0] +=
+ ((end - start) / PrimitiveArrayManager.ARRAY_SIZE + 1)
+ * TVList.vectorTvListArrayMemSize(dataTypes);
+ } else {
+ int acquireArray =
+ (end - start - 1 + (currentChunkPointNum % PrimitiveArrayManager.ARRAY_SIZE))
+ / PrimitiveArrayManager.ARRAY_SIZE;
+ memIncrements[0] +=
+ acquireArray == 0 ? 0 : acquireArray * TVList.vectorTvListArrayMemSize(dataTypes);
+ }
+ }
+ // TEXT data size
+ for (int i = 0; i < dataTypes.size(); i++) {
+ if (dataTypes.get(i) == TSDataType.TEXT) {
+ Binary[] binColumn = (Binary[]) columns[i];
+ memIncrements[1] += MemUtils.getBinaryColumnSize(binColumn, start, end);
+ }
+ }
+ }
+
private void updateMemoryInfo(
long memTableIncrement,
long unsealedResourceIncrement,
@@ -1055,16 +1130,12 @@
*
* @param deviceId device id
* @param measurementId measurements id
- * @param dataType data type
- * @param encoding encoding
*/
@SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
public void query(
String deviceId,
String measurementId,
- TSDataType dataType,
- TSEncoding encoding,
- Map<String, String> props,
+ IMeasurementSchema schema,
QueryContext context,
List<TsFileResource> tsfileResourcesForQuery)
throws IOException, MetadataException {
@@ -1086,13 +1157,7 @@
flushingMemTable, deviceId, measurementId, context.getQueryTimeLowerBound());
ReadOnlyMemChunk memChunk =
flushingMemTable.query(
- deviceId,
- measurementId,
- dataType,
- encoding,
- props,
- context.getQueryTimeLowerBound(),
- deletionList);
+ deviceId, measurementId, schema, context.getQueryTimeLowerBound(), deletionList);
if (memChunk != null) {
readOnlyMemChunks.add(memChunk);
}
@@ -1100,13 +1165,7 @@
if (workMemTable != null) {
ReadOnlyMemChunk memChunk =
workMemTable.query(
- deviceId,
- measurementId,
- dataType,
- encoding,
- props,
- context.getQueryTimeLowerBound(),
- null);
+ deviceId, measurementId, schema, context.getQueryTimeLowerBound(), null);
if (memChunk != null) {
readOnlyMemChunks.add(memChunk);
}
@@ -1118,8 +1177,33 @@
modificationFile,
new PartialPath(deviceId + IoTDBConstant.PATH_SEPARATOR + measurementId));
- List<ChunkMetadata> chunkMetadataList =
- writer.getVisibleMetadataList(deviceId, measurementId, dataType);
+ List<IChunkMetadata> chunkMetadataList = new ArrayList<>();
+ if (schema instanceof VectorMeasurementSchema) {
+ List<ChunkMetadata> timeChunkMetadataList =
+ writer.getVisibleMetadataList(deviceId, measurementId, schema.getType());
+ List<List<ChunkMetadata>> valueChunkMetadataList = new ArrayList<>();
+ List<String> valueMeasurementIdList = schema.getValueMeasurementIdList();
+ List<TSDataType> valueDataTypeList = schema.getValueTSDataTypeList();
+ for (int i = 0; i < valueMeasurementIdList.size(); i++) {
+ valueChunkMetadataList.add(
+ writer.getVisibleMetadataList(
+ deviceId, valueMeasurementIdList.get(i), valueDataTypeList.get(i)));
+ }
+
+ for (int i = 0; i < timeChunkMetadataList.size(); i++) {
+ List<IChunkMetadata> valueChunkMetadata = new ArrayList<>();
+ for (List<ChunkMetadata> chunkMetadata : valueChunkMetadataList) {
+ valueChunkMetadata.add(chunkMetadata.get(i));
+ }
+ chunkMetadataList.add(
+ new VectorChunkMetadata(timeChunkMetadataList.get(i), valueChunkMetadata));
+ }
+ } else {
+ chunkMetadataList =
+ new ArrayList<>(
+ writer.getVisibleMetadataList(deviceId, measurementId, schema.getType()));
+ }
+
QueryUtils.modifyChunkMetaData(chunkMetadataList, modifications);
chunkMetadataList.removeIf(context::chunkNotSatisfy);
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
index 9ff5217..c6a7abc 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
@@ -32,8 +32,11 @@
import org.apache.iotdb.db.utils.FilePathUtils;
import org.apache.iotdb.db.utils.TestOnly;
import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ITimeSeriesMetadata;
import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.file.metadata.VectorChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.VectorTimeSeriesMetadata;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
@@ -111,13 +114,13 @@
* Chunk metadata list of unsealed tsfile. Only be set in a temporal TsFileResource in a query
* process.
*/
- private List<ChunkMetadata> chunkMetadataList;
+ private List<IChunkMetadata> chunkMetadataList;
/** Mem chunk data. Only be set in a temporal TsFileResource in a query process. */
private List<ReadOnlyMemChunk> readOnlyMemChunk;
/** used for unsealed file to get TimeseriesMetadata */
- private TimeseriesMetadata timeSeriesMetadata;
+ private ITimeSeriesMetadata timeSeriesMetadata;
private FSFactory fsFactory = FSFactoryProducer.getFSFactory();
@@ -192,7 +195,7 @@
/** unsealed TsFile */
public TsFileResource(
List<ReadOnlyMemChunk> readOnlyMemChunk,
- List<ChunkMetadata> chunkMetadataList,
+ List<IChunkMetadata> chunkMetadataList,
TsFileResource originTsFileResource)
throws IOException {
this.file = originTsFileResource.file;
@@ -213,36 +216,106 @@
this.timeIndexType = 1;
}
+ @SuppressWarnings("squid:S3776") // high Cognitive Complexity
private void generateTimeSeriesMetadata() throws IOException {
- timeSeriesMetadata = new TimeseriesMetadata();
- timeSeriesMetadata.setOffsetOfChunkMetaDataList(-1);
- timeSeriesMetadata.setDataSizeOfChunkMetaDataList(-1);
+ TimeseriesMetadata timeTimeSeriesMetadata = new TimeseriesMetadata();
+ timeTimeSeriesMetadata.setOffsetOfChunkMetaDataList(-1);
+ timeTimeSeriesMetadata.setDataSizeOfChunkMetaDataList(-1);
if (!(chunkMetadataList == null || chunkMetadataList.isEmpty())) {
- timeSeriesMetadata.setMeasurementId(chunkMetadataList.get(0).getMeasurementUid());
+ timeTimeSeriesMetadata.setMeasurementId(chunkMetadataList.get(0).getMeasurementUid());
TSDataType dataType = chunkMetadataList.get(0).getDataType();
- timeSeriesMetadata.setTSDataType(dataType);
+ timeTimeSeriesMetadata.setTSDataType(dataType);
} else if (!(readOnlyMemChunk == null || readOnlyMemChunk.isEmpty())) {
- timeSeriesMetadata.setMeasurementId(readOnlyMemChunk.get(0).getMeasurementUid());
+ timeTimeSeriesMetadata.setMeasurementId(readOnlyMemChunk.get(0).getMeasurementUid());
TSDataType dataType = readOnlyMemChunk.get(0).getDataType();
- timeSeriesMetadata.setTSDataType(dataType);
+ timeTimeSeriesMetadata.setTSDataType(dataType);
}
- if (timeSeriesMetadata.getTSDataType() != null) {
- Statistics<?> seriesStatistics =
- Statistics.getStatsByType(timeSeriesMetadata.getTSDataType());
- // flush chunkMetadataList one by one
- for (ChunkMetadata chunkMetadata : chunkMetadataList) {
- seriesStatistics.mergeStatistics(chunkMetadata.getStatistics());
- }
+ if (timeTimeSeriesMetadata.getTSDataType() != null) {
+ if (timeTimeSeriesMetadata.getTSDataType() == TSDataType.VECTOR) {
+ Statistics<?> timeStatistics =
+ Statistics.getStatsByType(timeTimeSeriesMetadata.getTSDataType());
- for (ReadOnlyMemChunk memChunk : readOnlyMemChunk) {
- if (!memChunk.isEmpty()) {
- seriesStatistics.mergeStatistics(memChunk.getChunkMetaData().getStatistics());
+ List<TimeseriesMetadata> valueTimeSeriesMetadataList = new ArrayList<>();
+
+ if (!(chunkMetadataList == null || chunkMetadataList.isEmpty())) {
+ VectorChunkMetadata vectorChunkMetadata = (VectorChunkMetadata) chunkMetadataList.get(0);
+ for (IChunkMetadata valueChunkMetadata :
+ vectorChunkMetadata.getValueChunkMetadataList()) {
+ TimeseriesMetadata valueMetadata = new TimeseriesMetadata();
+ valueMetadata.setOffsetOfChunkMetaDataList(-1);
+ valueMetadata.setDataSizeOfChunkMetaDataList(-1);
+ valueMetadata.setMeasurementId(valueChunkMetadata.getMeasurementUid());
+ valueMetadata.setTSDataType(valueChunkMetadata.getDataType());
+ valueTimeSeriesMetadataList.add(valueMetadata);
+ valueMetadata.setStatistics(
+ Statistics.getStatsByType(valueChunkMetadata.getDataType()));
+ }
+ } else if (!(readOnlyMemChunk == null || readOnlyMemChunk.isEmpty())) {
+ VectorChunkMetadata vectorChunkMetadata =
+ (VectorChunkMetadata) readOnlyMemChunk.get(0).getChunkMetaData();
+ for (IChunkMetadata valueChunkMetadata :
+ vectorChunkMetadata.getValueChunkMetadataList()) {
+ TimeseriesMetadata valueMetadata = new TimeseriesMetadata();
+ valueMetadata.setOffsetOfChunkMetaDataList(-1);
+ valueMetadata.setDataSizeOfChunkMetaDataList(-1);
+ valueMetadata.setMeasurementId(valueChunkMetadata.getMeasurementUid());
+ valueMetadata.setTSDataType(valueChunkMetadata.getDataType());
+ valueTimeSeriesMetadataList.add(valueMetadata);
+ valueMetadata.setStatistics(
+ Statistics.getStatsByType(valueChunkMetadata.getDataType()));
+ }
}
+
+ for (IChunkMetadata chunkMetadata : chunkMetadataList) {
+ VectorChunkMetadata vectorChunkMetadata = (VectorChunkMetadata) chunkMetadata;
+ timeStatistics.mergeStatistics(
+ vectorChunkMetadata.getTimeChunkMetadata().getStatistics());
+ for (int i = 0; i < valueTimeSeriesMetadataList.size(); i++) {
+ valueTimeSeriesMetadataList
+ .get(i)
+ .getStatistics()
+ .mergeStatistics(
+ vectorChunkMetadata.getValueChunkMetadataList().get(i).getStatistics());
+ }
+ }
+
+ for (ReadOnlyMemChunk memChunk : readOnlyMemChunk) {
+ if (!memChunk.isEmpty()) {
+ VectorChunkMetadata vectorChunkMetadata =
+ (VectorChunkMetadata) memChunk.getChunkMetaData();
+ timeStatistics.mergeStatistics(
+ vectorChunkMetadata.getTimeChunkMetadata().getStatistics());
+ for (int i = 0; i < valueTimeSeriesMetadataList.size(); i++) {
+ valueTimeSeriesMetadataList
+ .get(i)
+ .getStatistics()
+ .mergeStatistics(
+ vectorChunkMetadata.getValueChunkMetadataList().get(i).getStatistics());
+ }
+ }
+ }
+ timeTimeSeriesMetadata.setStatistics(timeStatistics);
+ timeSeriesMetadata =
+ new VectorTimeSeriesMetadata(timeTimeSeriesMetadata, valueTimeSeriesMetadataList);
+ } else {
+ Statistics<?> seriesStatistics =
+ Statistics.getStatsByType(timeTimeSeriesMetadata.getTSDataType());
+ // flush chunkMetadataList one by one
+ for (IChunkMetadata chunkMetadata : chunkMetadataList) {
+ seriesStatistics.mergeStatistics(chunkMetadata.getStatistics());
+ }
+
+ for (ReadOnlyMemChunk memChunk : readOnlyMemChunk) {
+ if (!memChunk.isEmpty()) {
+ seriesStatistics.mergeStatistics(memChunk.getChunkMetaData().getStatistics());
+ }
+ }
+ timeTimeSeriesMetadata.setStatistics(seriesStatistics);
+ this.timeSeriesMetadata = timeTimeSeriesMetadata;
}
- timeSeriesMetadata.setStatistics(seriesStatistics);
} else {
- timeSeriesMetadata = null;
+ this.timeSeriesMetadata = null;
}
}
@@ -349,7 +422,7 @@
return fsFactory.getFile(file + RESOURCE_SUFFIX).exists();
}
- public List<ChunkMetadata> getChunkMetadataList() {
+ public List<IChunkMetadata> getChunkMetadataList() {
return new ArrayList<>(chunkMetadataList);
}
@@ -589,8 +662,24 @@
this.processor = processor;
}
+ /**
+ * Get a timeseriesMetadata.
+ *
+ * @return TimeseriesMetadata or the first ValueTimeseriesMetadata in VectorTimeseriesMetadata
+ */
public TimeseriesMetadata getTimeSeriesMetadata() {
- return timeSeriesMetadata;
+ if (timeSeriesMetadata == null) {
+ return null;
+ }
+ if (timeSeriesMetadata instanceof TimeseriesMetadata) {
+ return (TimeseriesMetadata) timeSeriesMetadata;
+ } else {
+ // it's ok for us to return the first value timeseries metadata,
+ // because the MemChunkMetadataLoader is not depend on the timeseries metadata
+ return ((VectorTimeSeriesMetadata) timeSeriesMetadata)
+ .getValueTimeseriesMetadataList()
+ .get(0);
+ }
}
public void setUpgradedResources(List<TsFileResource> upgradedResources) {
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/AlignedTimeseriesException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/AlignedTimeseriesException.java
new file mode 100644
index 0000000..f8373d7
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/AlignedTimeseriesException.java
@@ -0,0 +1,35 @@
+/*
+ * 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.exception.metadata;
+
+import org.apache.iotdb.rpc.TSStatusCode;
+
+public class AlignedTimeseriesException extends MetadataException {
+
+ /**
+ * @param message excepetion message
+ * @param path related timeseries path
+ */
+ public AlignedTimeseriesException(String message, String path) {
+ super(String.format("%s (Path: %s)", message, path));
+ errorCode = TSStatusCode.ALIGNED_TIMESERIES_ERROR.getStatusCode();
+ this.isUserException = true;
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/DuplicatedTemplateException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/DuplicatedTemplateException.java
new file mode 100644
index 0000000..0ffee81
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/DuplicatedTemplateException.java
@@ -0,0 +1,31 @@
+/*
+ * 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.exception.metadata;
+
+import org.apache.iotdb.rpc.TSStatusCode;
+
+public class DuplicatedTemplateException extends MetadataException {
+ public DuplicatedTemplateException(String path) {
+ super(
+ String.format("Failed to create duplicated template for path %s", path),
+ TSStatusCode.DUPLICATED_TEMPLATE.getStatusCode(),
+ true);
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/StorageGroupNotSetException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/StorageGroupNotSetException.java
index fb1b79b..aa38993 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/metadata/StorageGroupNotSetException.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/StorageGroupNotSetException.java
@@ -19,16 +19,20 @@
package org.apache.iotdb.db.exception.metadata;
+import org.apache.iotdb.rpc.TSStatusCode;
+
public class StorageGroupNotSetException extends MetadataException {
private static final long serialVersionUID = 3739300272099030533L;
public StorageGroupNotSetException(String path) {
super(String.format("Storage group is not set for current seriesPath: [%s]", path));
+ this.errorCode = TSStatusCode.STORAGE_GROUP_NOT_EXIST.getStatusCode();
}
public StorageGroupNotSetException(String path, boolean isUserException) {
super(String.format("Storage group is not set for current seriesPath: [%s]", path));
this.isUserException = isUserException;
+ this.errorCode = TSStatusCode.STORAGE_GROUP_NOT_EXIST.getStatusCode();
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/UndefinedTemplateException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/UndefinedTemplateException.java
new file mode 100644
index 0000000..32c2729
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/UndefinedTemplateException.java
@@ -0,0 +1,31 @@
+/*
+ * 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.exception.metadata;
+
+import org.apache.iotdb.rpc.TSStatusCode;
+
+public class UndefinedTemplateException extends MetadataException {
+ public UndefinedTemplateException(String path) {
+ super(
+ String.format("Undefined template name: " + path),
+ TSStatusCode.UNDEFINED_TEMPLATE.getStatusCode(),
+ true);
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MLogTxtWriter.java b/server/src/main/java/org/apache/iotdb/db/metadata/MLogTxtWriter.java
index ebfa2c6..f1bc0b6 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MLogTxtWriter.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MLogTxtWriter.java
@@ -19,12 +19,15 @@
package org.apache.iotdb.db.metadata;
import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+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.MNodePlan;
import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -41,6 +44,7 @@
import java.nio.channels.FileChannel;
import java.nio.file.Files;
import java.nio.file.Paths;
+import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
@@ -120,6 +124,34 @@
lineNumber.incrementAndGet();
}
+ public void createAlignedTimeseries(CreateAlignedTimeSeriesPlan plan) throws IOException {
+ StringBuilder buf = new StringBuilder();
+ buf.append(
+ String.format(
+ "%s,%s,%s,%s,%s,%s",
+ MetadataOperationType.CREATE_TIMESERIES,
+ plan.getDevicePath().getFullPath(),
+ plan.getMeasurements(),
+ plan.getDataTypes().stream().map(TSDataType::serialize),
+ plan.getEncodings().stream().map(TSEncoding::serialize),
+ plan.getCompressor().serialize()));
+
+ buf.append(",[");
+ if (plan.getAliasList() != null) {
+ List<String> aliasList = plan.getAliasList();
+ for (int i = 0; i < aliasList.size(); i++) {
+ buf.append(aliasList.get(i));
+ if (i != aliasList.size() - 1) {
+ buf.append(",");
+ }
+ }
+ }
+ buf.append("]");
+ buf.append(LINE_SEPARATOR);
+ channel.write(ByteBuffer.wrap(buf.toString().getBytes()));
+ lineNumber.incrementAndGet();
+ }
+
public void deleteTimeseries(String path) throws IOException {
String outputStr = MetadataOperationType.DELETE_TIMESERIES + "," + path + LINE_SEPARATOR;
ByteBuffer buff = ByteBuffer.wrap(outputStr.getBytes());
@@ -228,7 +260,7 @@
if (plan.getAlias() != null) {
s.append(plan.getAlias());
}
- MeasurementSchema schema = plan.getSchema();
+ IMeasurementSchema schema = plan.getSchema();
s.append(",").append(schema.getType().ordinal()).append(",");
s.append(schema.getEncodingType().ordinal()).append(",");
s.append(schema.getCompressor().ordinal()).append(",");
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
index 74617f7..21ae5a1 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
@@ -26,32 +26,41 @@
import org.apache.iotdb.db.engine.trigger.executor.TriggerEngine;
import org.apache.iotdb.db.exception.StorageEngineException;
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.DuplicatedTemplateException;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.exception.metadata.MetadataException;
import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
import org.apache.iotdb.db.exception.metadata.PathNotExistException;
import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
import org.apache.iotdb.db.metadata.logfile.MLogReader;
import org.apache.iotdb.db.metadata.logfile.MLogWriter;
import org.apache.iotdb.db.metadata.mnode.MNode;
import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.template.Template;
import org.apache.iotdb.db.monitor.MonitorConstants;
import org.apache.iotdb.db.qp.constant.SQLConstant;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
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.crud.SetDeviceTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.AutoCreateDeviceMNodePlan;
import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+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.DeleteStorageGroupPlan;
import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetUsingDeviceTemplatePlan;
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;
@@ -62,7 +71,6 @@
import org.apache.iotdb.db.utils.RandomDeleteCache;
import org.apache.iotdb.db.utils.SchemaUtils;
import org.apache.iotdb.db.utils.TestOnly;
-import org.apache.iotdb.db.utils.TypeInferenceUtils;
import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
import org.apache.iotdb.tsfile.exception.cache.CacheException;
import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
@@ -70,8 +78,10 @@
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.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -81,12 +91,14 @@
import java.nio.file.Files;
import java.util.ArrayDeque;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.Deque;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.LinkedHashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
@@ -137,7 +149,7 @@
private TagLogFile tagLogFile;
private boolean isRecovering;
// device -> DeviceMNode
- private RandomDeleteCache<PartialPath, MNode> mNodeCache;
+ private RandomDeleteCache<PartialPath, Pair<MNode, Template>> mNodeCache;
// tag key -> tag value -> LeafMNode
private Map<String, Map<String, Set<MeasurementMNode>>> tagIndex = new ConcurrentHashMap<>();
@@ -160,6 +172,9 @@
private static final int ESTIMATED_SERIES_SIZE = config.getEstimatedSeriesSize();
+ // template name -> template
+ private Map<String, Template> templateMap = new ConcurrentHashMap<>();
+
private static class MManagerHolder {
private MManagerHolder() {
@@ -190,10 +205,10 @@
int cacheSize = config.getmManagerCacheSize();
mNodeCache =
- new RandomDeleteCache<PartialPath, MNode>(cacheSize) {
+ new RandomDeleteCache<PartialPath, Pair<MNode, Template>>(cacheSize) {
@Override
- public MNode loadObjectByKey(PartialPath key) throws CacheException {
+ public Pair<MNode, Template> loadObjectByKey(PartialPath key) throws CacheException {
try {
return mtree.getNodeByPathWithStorageGroupCheck(key);
} catch (MetadataException e) {
@@ -308,10 +323,13 @@
/** function for clearing MTree */
public void clear() {
try {
+ templateMap.clear();
+ Template.clear();
this.mtree = new MTree();
this.mNodeCache.clear();
this.tagIndex.clear();
this.totalSeriesNumber.set(0);
+ this.templateMap.clear();
if (logWriter != null) {
logWriter.close();
logWriter = null;
@@ -342,6 +360,11 @@
CreateTimeSeriesPlan createTimeSeriesPlan = (CreateTimeSeriesPlan) plan;
createTimeseries(createTimeSeriesPlan, createTimeSeriesPlan.getTagOffset());
break;
+ case CREATE_ALIGNED_TIMESERIES:
+ CreateAlignedTimeSeriesPlan createAlignedTimeSeriesPlan =
+ (CreateAlignedTimeSeriesPlan) plan;
+ createAlignedTimeSeries(createAlignedTimeSeriesPlan);
+ break;
case DELETE_TIMESERIES:
DeleteTimeSeriesPlan deleteTimeSeriesPlan = (DeleteTimeSeriesPlan) plan;
// cause we only has one path for one DeleteTimeSeriesPlan
@@ -367,6 +390,22 @@
ChangeTagOffsetPlan changeTagOffsetPlan = (ChangeTagOffsetPlan) plan;
changeOffset(changeTagOffsetPlan.getPath(), changeTagOffsetPlan.getOffset());
break;
+ case CREATE_TEMPLATE:
+ CreateTemplatePlan createTemplatePlan = (CreateTemplatePlan) plan;
+ createDeviceTemplate(createTemplatePlan);
+ break;
+ case SET_DEVICE_TEMPLATE:
+ SetDeviceTemplatePlan setDeviceTemplatePlan = (SetDeviceTemplatePlan) plan;
+ setDeviceTemplate(setDeviceTemplatePlan);
+ break;
+ case SET_USING_DEVICE_TEMPLATE:
+ SetUsingDeviceTemplatePlan setUsingDeviceTemplatePlan = (SetUsingDeviceTemplatePlan) plan;
+ setUsingDeviceTemplate(setUsingDeviceTemplatePlan);
+ break;
+ case AUTO_CREATE_DEVICE_MNODE:
+ AutoCreateDeviceMNodePlan autoCreateDeviceMNodePlan = (AutoCreateDeviceMNodePlan) plan;
+ autoCreateDeviceMNode(autoCreateDeviceMNodePlan);
+ break;
default:
logger.error("Unrecognizable command {}", plan.getOperatorType());
}
@@ -480,12 +519,69 @@
}
}
+ public void createAlignedTimeSeries(
+ PartialPath devicePath,
+ List<String> measurements,
+ List<TSDataType> dataTypes,
+ List<TSEncoding> encodings,
+ CompressionType compressor)
+ throws MetadataException {
+ createAlignedTimeSeries(
+ new CreateAlignedTimeSeriesPlan(
+ devicePath, measurements, dataTypes, encodings, compressor, null));
+ }
+
+ /**
+ * create aligned timeseries
+ *
+ * @param plan CreateAlignedTimeSeriesPlan
+ */
+ public void createAlignedTimeSeries(CreateAlignedTimeSeriesPlan plan) throws MetadataException {
+ if (!allowToCreateNewSeries) {
+ throw new MetadataException(
+ "IoTDB system load is too large to create timeseries, "
+ + "please increase MAX_HEAP_SIZE in iotdb-env.sh/bat and restart");
+ }
+ try {
+ PartialPath devicePath = plan.getDevicePath();
+ List<String> measurements = plan.getMeasurements();
+ int alignedSize = measurements.size();
+ List<TSDataType> dataTypes = plan.getDataTypes();
+ List<TSEncoding> encodings = plan.getEncodings();
+
+ for (int i = 0; i < alignedSize; i++) {
+ SchemaUtils.checkDataTypeWithEncoding(dataTypes.get(i), encodings.get(i));
+ }
+
+ ensureStorageGroup(devicePath);
+
+ // create time series in MTree
+ mtree.createAlignedTimeseries(
+ devicePath, measurements, plan.getDataTypes(), plan.getEncodings(), plan.getCompressor());
+
+ // update statistics and schemaDataTypeNumMap
+ totalSeriesNumber.addAndGet(measurements.size());
+ if (totalSeriesNumber.get() * ESTIMATED_SERIES_SIZE >= MTREE_SIZE_THRESHOLD) {
+ logger.warn("Current series number {} is too large...", totalSeriesNumber);
+ allowToCreateNewSeries = false;
+ }
+ for (TSDataType type : dataTypes) {
+ updateSchemaDataTypeNumMap(type, 1);
+ }
+ // write log
+ if (!isRecovering) {
+ logWriter.createAlignedTimeseries(plan);
+ }
+ } catch (IOException e) {
+ throw new MetadataException(e);
+ }
+ }
+
/**
* Delete all timeseries under the given path, may cross different storage group
*
- * @param prefixPath path to be deleted, could be root or a prefix path or a full path TODO:
- * directly return the failed string set
- * @return The String is the deletion failed Timeseries
+ * @param prefixPath path to be deleted, could be root or a prefix path or a full path
+ * @return deletion failed Timeseries
*/
public String deleteTimeseries(PartialPath prefixPath) throws MetadataException {
if (isStorageGroup(prefixPath)) {
@@ -496,6 +592,22 @@
if (allTimeseries.isEmpty()) {
throw new PathNotExistException(prefixPath.getFullPath());
}
+
+ // for not support deleting part of aligned timeseies
+ // should be removed after partial deletion is supported
+ MNode lastNode = getNodeByPath(allTimeseries.get(0));
+ if (lastNode instanceof MeasurementMNode) {
+ IMeasurementSchema schema = ((MeasurementMNode) lastNode).getSchema();
+ if (schema instanceof VectorMeasurementSchema) {
+ if (schema.getValueMeasurementIdList().size() != allTimeseries.size()) {
+ throw new AlignedTimeseriesException(
+ "Not support deleting part of aligned timeseies!", prefixPath.getFullPath());
+ } else {
+ allTimeseries.add(lastNode.getPartialPath());
+ }
+ }
+ }
+
// Monitor storage group seriesPath is not allowed to be deleted
allTimeseries.removeIf(p -> p.startsWith(MonitorConstants.STAT_STORAGE_GROUP_ARRAY));
@@ -572,13 +684,25 @@
/**
* @param path full path from root to leaf node
- * @return after delete if the storage group is empty, return its path, otherwise return null
+ * @return After delete if the storage group is empty, return its path, otherwise return null
*/
private PartialPath deleteOneTimeseriesUpdateStatisticsAndDropTrigger(PartialPath path)
throws MetadataException, IOException {
Pair<PartialPath, MeasurementMNode> pair =
mtree.deleteTimeseriesAndReturnEmptyStorageGroup(path);
- removeFromTagInvertedIndex(pair.right);
+ // if one of the aligned timeseries is deleted, pair.right could be null
+ IMeasurementSchema schema = pair.right.getSchema();
+ int timeseriesNum = 0;
+ if (schema instanceof MeasurementSchema) {
+ removeFromTagInvertedIndex(pair.right);
+ updateSchemaDataTypeNumMap(schema.getType(), -1);
+ timeseriesNum = 1;
+ } else if (schema instanceof VectorMeasurementSchema) {
+ for (TSDataType dataType : schema.getValueTSDataTypeList()) {
+ updateSchemaDataTypeNumMap(dataType, -1);
+ timeseriesNum++;
+ }
+ }
PartialPath storageGroupPath = pair.left;
// update statistics in schemaDataTypeNumMap
@@ -589,7 +713,7 @@
// TODO: delete the path node and all its ancestors
mNodeCache.clear();
- totalSeriesNumber.addAndGet(-1);
+ totalSeriesNumber.addAndGet(-timeseriesNum);
if (!allowToCreateNewSeries
&& totalSeriesNumber.get() * ESTIMATED_SERIES_SIZE < MTREE_SIZE_THRESHOLD) {
logger.info("Current series number {} come back to normal level", totalSeriesNumber);
@@ -704,7 +828,20 @@
return TSDataType.INT64;
}
- return mtree.getSchema(path).getType();
+ if (path instanceof VectorPartialPath) {
+ if (((VectorPartialPath) path).getSubSensorsPathList().size() != 1) {
+ return TSDataType.VECTOR;
+ } else {
+ path = ((VectorPartialPath) path).getSubSensorsPathList().get(0);
+ }
+ }
+ IMeasurementSchema schema = mtree.getSchema(path);
+ if (schema instanceof MeasurementSchema) {
+ return schema.getType();
+ } else {
+ List<String> measurements = schema.getValueMeasurementIdList();
+ return schema.getValueTSDataTypeList().get(measurements.indexOf(path.getMeasurement()));
+ }
}
public MeasurementMNode[] getMNodes(PartialPath deviceId, String[] measurements)
@@ -915,7 +1052,7 @@
try {
Pair<Map<String, String>, Map<String, String>> tagAndAttributePair =
tagLogFile.read(config.getTagAttributeTotalSize(), leaf.getOffset());
- MeasurementSchema measurementSchema = leaf.getSchema();
+ IMeasurementSchema measurementSchema = leaf.getSchema();
res.add(
new ShowTimeSeriesResult(
leaf.getFullPath(),
@@ -1003,14 +1140,152 @@
return res;
}
- public MeasurementSchema getSeriesSchema(PartialPath device, String measurement)
+ /**
+ * get MeasurementSchema or VectorMeasurementSchema which contains the measurement
+ *
+ * @param device device path
+ * @param measurement measurement name, could start with "$#$"
+ * @return MeasurementSchema or VectorMeasurementSchema
+ */
+ public IMeasurementSchema getSeriesSchema(PartialPath device, String measurement)
throws MetadataException {
- MNode node = mtree.getNodeByPath(device);
- MNode leaf = node.getChild(measurement);
- if (leaf != null) {
- return ((MeasurementMNode) leaf).getSchema();
+ MNode deviceMNode = getDeviceNode(device);
+ MeasurementMNode measurementMNode = (MeasurementMNode) deviceMNode.getChild(measurement);
+ return measurementMNode.getSchema();
+ }
+
+ /**
+ * Get schema of paritialPath
+ *
+ * @param fullPath (may be ParitialPath or VectorPartialPath)
+ * @return MeasurementSchema or VectorMeasurementSchema
+ */
+ public IMeasurementSchema getSeriesSchema(PartialPath fullPath) throws MetadataException {
+ MeasurementMNode leaf = (MeasurementMNode) mtree.getNodeByPath(fullPath);
+ return getSeriesSchema(fullPath, leaf);
+ }
+
+ protected IMeasurementSchema getSeriesSchema(PartialPath fullPath, MeasurementMNode leaf) {
+ IMeasurementSchema schema = leaf.getSchema();
+
+ if (schema == null || schema.getType() != TSDataType.VECTOR) {
+ return schema;
}
- return null;
+ List<String> measurementsInLeaf = schema.getValueMeasurementIdList();
+ List<PartialPath> measurements = ((VectorPartialPath) fullPath).getSubSensorsPathList();
+ TSDataType[] types = new TSDataType[measurements.size()];
+ TSEncoding[] encodings = new TSEncoding[measurements.size()];
+
+ for (int i = 0; i < measurements.size(); i++) {
+ int index = measurementsInLeaf.indexOf(measurements.get(i).getMeasurement());
+ types[i] = schema.getValueTSDataTypeList().get(index);
+ encodings[i] = schema.getValueTSEncodingList().get(index);
+ }
+ String[] array = new String[measurements.size()];
+ for (int i = 0; i < array.length; i++) {
+ array[i] = measurements.get(i).getMeasurement();
+ }
+ return new VectorMeasurementSchema(
+ schema.getMeasurementId(), array, types, encodings, schema.getCompressor());
+ }
+
+ /**
+ * transform the PartialPath to VectorPartialPath if it is a sub sensor of one vector otherwise,
+ * we don't change it.
+ */
+ public PartialPath transformPath(PartialPath partialPath) throws MetadataException {
+ MeasurementMNode node = (MeasurementMNode) getNodeByPath(partialPath);
+ if (node.getSchema() instanceof MeasurementSchema) {
+ return partialPath;
+ } else {
+ List<PartialPath> subSensorsPathList = new ArrayList<>();
+ subSensorsPathList.add(partialPath);
+ return new VectorPartialPath(
+ partialPath.getDevice() + "." + node.getName(), subSensorsPathList);
+ }
+ }
+
+ /**
+ * Get schema of partialPaths, in which aligned timeseries should only organized to one schema.
+ * This method should be called when logical plan converts to physical plan.
+ *
+ * @param fullPaths full path list without pointing out which timeseries are aligned. For example,
+ * maybe (s1,s2) are aligned, but the input could be [root.sg1.d1.s1, root.sg1.d1.s2]
+ * @return Pair<List < PartialPath>, List<Integer>>. Size of partial path list could NOT equal to
+ * the input list size. For example, the VectorMeasurementSchema (s1,s2) would be returned
+ * once; Size of integer list must equal to the input list size. It indicates the index of
+ * elements of original list in the result list
+ */
+ public Pair<List<PartialPath>, Map<String, Integer>> getSeriesSchemas(List<PartialPath> fullPaths)
+ throws MetadataException {
+ Map<MNode, PartialPath> nodeToPartialPath = new LinkedHashMap<>();
+ Map<MNode, List<Integer>> nodeToIndex = new LinkedHashMap<>();
+ for (int i = 0; i < fullPaths.size(); i++) {
+ PartialPath path = fullPaths.get(i);
+ // use dfs to collect paths
+ MeasurementMNode node = (MeasurementMNode) getNodeByPath(path);
+ getNodeToPartialPath(node, nodeToPartialPath, nodeToIndex, path, i);
+ }
+ return getPair(fullPaths, nodeToPartialPath, nodeToIndex);
+ }
+
+ protected void getNodeToPartialPath(
+ MeasurementMNode node,
+ Map<MNode, PartialPath> nodeToPartialPath,
+ Map<MNode, List<Integer>> nodeToIndex,
+ PartialPath path,
+ int index)
+ throws MetadataException {
+ if (!nodeToPartialPath.containsKey(node)) {
+ if (node.getSchema() instanceof MeasurementSchema) {
+ nodeToPartialPath.put(node, path);
+ } else {
+ List<PartialPath> subSensorsPathList = new ArrayList<>();
+ subSensorsPathList.add(path);
+ nodeToPartialPath.put(
+ node,
+ new VectorPartialPath(path.getDevice() + "." + node.getName(), subSensorsPathList));
+ }
+ nodeToIndex.computeIfAbsent(node, k -> new ArrayList<>()).add(index);
+ } else {
+ // if nodeToPartialPath contains node
+ String existPath = nodeToPartialPath.get(node).getFullPath();
+ if (existPath.equals(path.getFullPath())) {
+ // could be the same path in different aggregate functions
+ nodeToIndex.get(node).add(index);
+ } else {
+ // could be VectorPartialPath
+ ((VectorPartialPath) nodeToPartialPath.get(node)).addSubSensor(path);
+ nodeToIndex.get(node).add(index);
+ }
+ }
+ }
+
+ protected Pair<List<PartialPath>, Map<String, Integer>> getPair(
+ List<PartialPath> fullPaths,
+ Map<MNode, PartialPath> nodeToPartialPath,
+ Map<MNode, List<Integer>> nodeToIndex)
+ throws MetadataException {
+ Map<String, Integer> indexMap = new HashMap<>();
+ int i = 0;
+ for (List<Integer> indexList : nodeToIndex.values()) {
+ for (int index : indexList) {
+ PartialPath partialPath = fullPaths.get(i);
+ if (indexMap.containsKey(partialPath.getFullPath())) {
+ throw new MetadataException(
+ "Query for measurement and its alias at the same time!", true);
+ }
+ indexMap.put(partialPath.getFullPath(), index);
+ if (partialPath.isMeasurementAliasExists()) {
+ indexMap.put(partialPath.getFullPathWithAlias(), index);
+ }
+ if (partialPath.isTsAliasExists()) {
+ indexMap.put(partialPath.getTsAlias(), index);
+ }
+ i++;
+ }
+ }
+ return new Pair<>(new ArrayList<>(nodeToPartialPath.values()), indexMap);
}
/**
@@ -1074,9 +1349,10 @@
*
* @param path path
*/
- public MNode getDeviceNodeWithAutoCreate(PartialPath path, boolean autoCreateSchema, int sgLevel)
- throws MetadataException {
- MNode node;
+ public Pair<MNode, Template> getDeviceNodeWithAutoCreate(
+ PartialPath path, boolean autoCreateSchema, boolean allowCreateSg, int sgLevel)
+ throws IOException, MetadataException {
+ Pair<MNode, Template> node;
boolean shouldSetStorageGroup;
try {
node = mNodeCache.get(path);
@@ -1090,28 +1366,65 @@
try {
if (shouldSetStorageGroup) {
- PartialPath storageGroupPath = MetaUtils.getStorageGroupPathByLevel(path, sgLevel);
- setStorageGroup(storageGroupPath);
+ if (allowCreateSg) {
+ PartialPath storageGroupPath = MetaUtils.getStorageGroupPathByLevel(path, sgLevel);
+ setStorageGroup(storageGroupPath);
+ } else {
+ throw new StorageGroupNotSetException(path.getFullPath());
+ }
}
node = mtree.getDeviceNodeWithAutoCreating(path, sgLevel);
+ if (!(node.left instanceof StorageGroupMNode)) {
+ logWriter.autoCreateDeviceMNode(new AutoCreateDeviceMNodePlan(node.left.getPartialPath()));
+ }
return node;
} catch (StorageGroupAlreadySetException e) {
// ignore set storage group concurrently
node = mtree.getDeviceNodeWithAutoCreating(path, sgLevel);
+ if (!(node.left instanceof StorageGroupMNode)) {
+ logWriter.autoCreateDeviceMNode(new AutoCreateDeviceMNodePlan(node.left.getPartialPath()));
+ }
return node;
}
}
/** !!!!!!Attention!!!!! must call the return node's readUnlock() if you call this method. */
- public MNode getDeviceNodeWithAutoCreate(PartialPath path) throws MetadataException {
+ public Pair<MNode, Template> getDeviceNodeWithAutoCreate(PartialPath path)
+ throws MetadataException, IOException {
return getDeviceNodeWithAutoCreate(
- path, config.isAutoCreateSchemaEnabled(), config.getDefaultStorageGroupLevel());
+ path, config.isAutoCreateSchemaEnabled(), true, config.getDefaultStorageGroupLevel());
+ }
+
+ // attention: this path must be a device node
+ public List<IMeasurementSchema> getAllMeasurementByDevicePath(PartialPath path)
+ throws PathNotExistException {
+ Set<IMeasurementSchema> res = new HashSet<>();
+ try {
+ Pair<MNode, Template> mNodeTemplatePair = mNodeCache.get(path);
+ if (mNodeTemplatePair.left.getDeviceTemplate() != null) {
+ mNodeTemplatePair.right = mNodeTemplatePair.left.getDeviceTemplate();
+ }
+
+ for (MNode mNode : mNodeTemplatePair.left.getChildren().values()) {
+ MeasurementMNode measurementMNode = (MeasurementMNode) mNode;
+ res.add(measurementMNode.getSchema());
+ }
+
+ // template
+ if (mNodeTemplatePair.left.isUseTemplate() && mNodeTemplatePair.right != null) {
+ res.addAll(mNodeTemplatePair.right.getSchemaMap().values());
+ }
+ } catch (CacheException e) {
+ throw new PathNotExistException(path.getFullPath());
+ }
+
+ return new ArrayList<>(res);
}
public MNode getDeviceNode(PartialPath path) throws MetadataException {
MNode node;
try {
- node = mNodeCache.get(path);
+ node = mNodeCache.get(path).left;
return node;
} catch (CacheException e) {
throw new PathNotExistException(path.getFullPath());
@@ -1661,7 +1974,7 @@
while (!nodeDeque.isEmpty()) {
MNode node = nodeDeque.removeFirst();
if (node instanceof MeasurementMNode) {
- MeasurementSchema nodeSchema = ((MeasurementMNode) node).getSchema();
+ IMeasurementSchema nodeSchema = ((MeasurementMNode) node).getSchema();
timeseriesSchemas.add(
new TimeseriesSchema(
node.getFullPath(),
@@ -1680,19 +1993,14 @@
}
public void collectMeasurementSchema(
- MNode startingNode, Collection<MeasurementSchema> measurementSchemas) {
+ MNode startingNode, Collection<IMeasurementSchema> measurementSchemas) {
Deque<MNode> nodeDeque = new ArrayDeque<>();
nodeDeque.addLast(startingNode);
while (!nodeDeque.isEmpty()) {
MNode node = nodeDeque.removeFirst();
if (node instanceof MeasurementMNode) {
- MeasurementSchema nodeSchema = ((MeasurementMNode) node).getSchema();
- measurementSchemas.add(
- new MeasurementSchema(
- node.getName(),
- nodeSchema.getType(),
- nodeSchema.getEncodingType(),
- nodeSchema.getCompressor()));
+ IMeasurementSchema nodeSchema = ((MeasurementMNode) node).getSchema();
+ measurementSchemas.add(nodeSchema);
} else if (!node.getChildren().isEmpty()) {
nodeDeque.addAll(node.getChildren().values());
}
@@ -1700,7 +2008,7 @@
}
/** Collect the timeseries schemas under "startingPath". */
- public void collectSeries(PartialPath startingPath, List<MeasurementSchema> measurementSchemas) {
+ public void collectSeries(PartialPath startingPath, List<IMeasurementSchema> measurementSchemas) {
MNode mNode;
try {
mNode = getNodeByPath(startingPath);
@@ -1839,52 +2147,98 @@
/** get schema for device. Attention!!! Only support insertPlan */
@SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
- public MNode getSeriesSchemasAndReadLockDevice(InsertPlan plan) throws MetadataException {
-
+ public MNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
+ throws MetadataException, IOException {
PartialPath deviceId = plan.getDeviceId();
String[] measurementList = plan.getMeasurements();
MeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
// 1. get device node
- MNode deviceMNode = getDeviceNodeWithAutoCreate(deviceId);
+ Pair<MNode, Template> deviceMNode = getDeviceNodeWithAutoCreate(deviceId);
+ if (deviceMNode.left.getDeviceTemplate() != null) {
+ deviceMNode.right = deviceMNode.left.getDeviceTemplate();
+ }
// 2. get schema of each measurement
- // if do not has measurement
+ // if do not have measurement
MeasurementMNode measurementMNode;
- TSDataType dataType;
+ int loc = 0;
+
for (int i = 0; i < measurementList.length; i++) {
try {
- MNode child = getMNode(deviceMNode, measurementList[i]);
+ String measurement = measurementList[i];
+ boolean isVector = false;
+ String firstMeasurementOfVector = null;
+ if (measurement.contains("(") && measurement.contains(",")) {
+ isVector = true;
+ firstMeasurementOfVector = measurement.replace("(", "").replace(")", "").split(",")[0];
+ }
+
+ MNode child = getMNode(deviceMNode.left, isVector ? firstMeasurementOfVector : measurement);
if (child instanceof MeasurementMNode) {
measurementMNode = (MeasurementMNode) child;
} else if (child instanceof StorageGroupMNode) {
- throw new PathAlreadyExistException(deviceId + PATH_SEPARATOR + measurementList[i]);
+ throw new PathAlreadyExistException(deviceId + PATH_SEPARATOR + measurement);
+ } else if ((measurementMNode = findTemplate(deviceMNode, measurement)) != null) {
+ // empty
} else {
if (!config.isAutoCreateSchemaEnabled()) {
- throw new PathNotExistException(deviceId + PATH_SEPARATOR + measurementList[i]);
+ throw new PathNotExistException(deviceId + PATH_SEPARATOR + measurement);
} else {
- // child is null or child is type of MNode
- dataType = getTypeInLoc(plan, i);
- // create it, may concurrent created by multiple thread
- internalCreateTimeseries(deviceId.concatNode(measurementList[i]), dataType);
- measurementMNode = (MeasurementMNode) deviceMNode.getChild(measurementList[i]);
+ if (plan instanceof InsertRowPlan || plan instanceof InsertTabletPlan) {
+ List<String> measurements =
+ Arrays.asList(measurement.replace("(", "").replace(")", "").split(","));
+ if (measurements.size() == 1) {
+ internalCreateTimeseries(
+ deviceId.concatNode(measurement), plan.getDataTypes()[loc++]);
+ measurementMNode = (MeasurementMNode) deviceMNode.left.getChild(measurement);
+
+ } else {
+ int curLoc = loc;
+ List<TSDataType> dataTypes = new ArrayList<>();
+ for (int j = 0; j < measurements.size(); j++) {
+ dataTypes.add(plan.getDataTypes()[curLoc]);
+ curLoc++;
+ }
+ internalAlignedCreateTimeseries(deviceId, measurements, dataTypes);
+ measurementMNode =
+ (MeasurementMNode) deviceMNode.left.getChild(measurements.get(0));
+ }
+ } else {
+ throw new MetadataException(
+ String.format(
+ "Only support insertRow and insertTablet, plan is [%s]",
+ plan.getOperatorType()));
+ }
}
}
// check type is match
+ boolean mismatch = false;
TSDataType insertDataType = null;
- if (plan instanceof InsertRowPlan) {
- if (!((InsertRowPlan) plan).isNeedInferType()) {
- // only when InsertRowPlan's values is object[], we should check type
- insertDataType = getTypeInLoc(plan, i);
+ if (plan instanceof InsertRowPlan || plan instanceof InsertTabletPlan) {
+ if (measurementList[i].contains("(") && measurementList[i].contains(",")) {
+ for (int j = 0; j < measurementList[i].split(",").length; j++) {
+ TSDataType dataTypeInNode =
+ measurementMNode.getSchema().getValueTSDataTypeList().get(j);
+ insertDataType = plan.getDataTypes()[loc];
+ if (insertDataType == null) {
+ insertDataType = dataTypeInNode;
+ }
+ if (dataTypeInNode != insertDataType) {
+ mismatch = true;
+ insertDataType = dataTypeInNode;
+ break;
+ }
+ loc++;
+ }
} else {
insertDataType = measurementMNode.getSchema().getType();
+ mismatch = measurementMNode.getSchema().getType() != insertDataType;
}
- } else if (plan instanceof InsertTabletPlan) {
- insertDataType = getTypeInLoc(plan, i);
}
- if (measurementMNode.getSchema().getType() != insertDataType) {
+ if (mismatch) {
logger.warn(
"DataType mismatch, Insert measurement {} type {}, metadata tree type {}",
measurementList[i],
@@ -1906,7 +2260,6 @@
// set measurementName instead of alias
measurementList[i] = measurementMNode.getName();
-
} catch (MetadataException e) {
logger.warn(
"meet error when check {}.{}, message: {}",
@@ -1922,14 +2275,48 @@
}
}
- return deviceMNode;
+ return deviceMNode.left;
}
public MNode getMNode(MNode deviceMNode, String measurementName) {
return deviceMNode.getChild(measurementName);
}
- /** create timeseries with ignore PathAlreadyExistException */
+ private MeasurementMNode findTemplate(Pair<MNode, Template> deviceMNode, String measurement)
+ throws MetadataException {
+ if (deviceMNode.right != null) {
+ Map<String, IMeasurementSchema> curTemplateMap = deviceMNode.right.getSchemaMap();
+ List<String> measurements =
+ Arrays.asList(measurement.replace("(", "").replace(")", "").split(","));
+
+ String firstMeasurement = measurements.get(0);
+ IMeasurementSchema schema = curTemplateMap.get(firstMeasurement);
+ if (!deviceMNode.left.isUseTemplate()) {
+ deviceMNode.left.setUseTemplate(true);
+ try {
+ logWriter.setUsingDeviceTemplate(deviceMNode.left.getPartialPath());
+ } catch (IOException e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ if (schema != null) {
+ if (schema instanceof MeasurementSchema) {
+ return new MeasurementMNode(deviceMNode.left, firstMeasurement, schema, null);
+ } else if (schema instanceof VectorMeasurementSchema) {
+ return new MeasurementMNode(
+ deviceMNode.left,
+ deviceMNode.right.getMeasurementNodeName(schema.getValueMeasurementIdList().get(0)),
+ schema,
+ null);
+ }
+ }
+ return null;
+ }
+ return null;
+ }
+
+ /** create timeseries ignoring PathAlreadyExistException */
private void internalCreateTimeseries(PartialPath path, TSDataType dataType)
throws MetadataException {
createTimeseries(
@@ -1940,21 +2327,20 @@
Collections.emptyMap());
}
- /** get dataType of plan, in loc measurements only support InsertRowPlan and InsertTabletPlan */
- private 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()));
+ /** create aligned timeseries ignoring PathAlreadyExistException */
+ private void internalAlignedCreateTimeseries(
+ PartialPath devicePath, List<String> measurements, List<TSDataType> dataTypes)
+ throws MetadataException {
+ List<TSEncoding> encodings = new ArrayList<>();
+ for (TSDataType dataType : dataTypes) {
+ encodings.add(getDefaultEncoding(dataType));
}
- return dataType;
+ createAlignedTimeSeries(
+ devicePath,
+ measurements,
+ dataTypes,
+ encodings,
+ TSFileDescriptor.getInstance().getConfig().getCompressor());
}
/**
@@ -1966,4 +2352,98 @@
boolean satisfy(String storageGroup);
}
+
+ public void createDeviceTemplate(CreateTemplatePlan plan) throws MetadataException {
+ try {
+ Template template = new Template(plan);
+ if (templateMap.putIfAbsent(plan.getName(), template) != null) {
+ // already have template
+ throw new MetadataException("Duplicated template name: " + plan.getName());
+ }
+
+ // write wal
+ if (!isRecovering) {
+ logWriter.createDeviceTemplate(plan);
+ }
+ } catch (IOException e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ public void setDeviceTemplate(SetDeviceTemplatePlan plan) throws MetadataException {
+ try {
+ Template template = templateMap.get(plan.getTemplateName());
+
+ if (template == null) {
+ throw new UndefinedTemplateException(plan.getTemplateName());
+ }
+
+ // get mnode and update template should be atomic
+ synchronized (this) {
+ Pair<MNode, Template> node =
+ getDeviceNodeWithAutoCreate(new PartialPath(plan.getPrefixPath()));
+
+ if (node.left.getDeviceTemplate() != null) {
+ if (node.left.getDeviceTemplate().equals(template)) {
+ throw new DuplicatedTemplateException(template.getName());
+ } else {
+ throw new MetadataException("Specified node already has template");
+ }
+ }
+
+ if (!isTemplateCompatible(node.right, template)) {
+ throw new MetadataException("Incompatible template");
+ }
+
+ node.left.setDeviceTemplate(template);
+ }
+
+ // write wal
+ if (!isRecovering) {
+ logWriter.setDeviceTemplate(plan);
+ }
+ } catch (IOException e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ public boolean isTemplateCompatible(Template upper, Template current) {
+ if (upper == null) {
+ return true;
+ }
+
+ Map<String, IMeasurementSchema> upperMap = new HashMap<>(upper.getSchemaMap());
+ Map<String, IMeasurementSchema> currentMap = new HashMap<>(current.getSchemaMap());
+
+ // for identical vector schema, we should just compare once
+ Map<IMeasurementSchema, IMeasurementSchema> sameSchema = new HashMap<>();
+
+ for (String name : currentMap.keySet()) {
+ IMeasurementSchema upperSchema = upperMap.remove(name);
+ if (upperSchema != null) {
+ IMeasurementSchema currentSchema = currentMap.get(name);
+ // use "==" to compare actual address space
+ if (upperSchema == sameSchema.get(currentSchema)) {
+ continue;
+ }
+
+ if (!upperSchema.equals(currentSchema)) {
+ return false;
+ }
+
+ sameSchema.put(currentSchema, upperSchema);
+ }
+ }
+
+ // current template must contains all measurements of upper template
+ return upperMap.isEmpty();
+ }
+
+ public void autoCreateDeviceMNode(AutoCreateDeviceMNodePlan plan) throws MetadataException {
+ mtree.getDeviceNodeWithAutoCreating(plan.getPath(), config.getDefaultStorageGroupLevel());
+ }
+
+ private void setUsingDeviceTemplate(SetUsingDeviceTemplatePlan plan) throws MetadataException {
+ getDeviceNode(plan.getPrefixPath()).setUseTemplate(true);
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
index 39a5088..3c21e8c 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
@@ -22,15 +22,27 @@
import org.apache.iotdb.db.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
-import org.apache.iotdb.db.exception.metadata.*;
+import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.IllegalParameterOfPathException;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
import org.apache.iotdb.db.metadata.MManager.StorageGroupFilter;
import org.apache.iotdb.db.metadata.logfile.MLogReader;
import org.apache.iotdb.db.metadata.logfile.MLogWriter;
import org.apache.iotdb.db.metadata.mnode.MNode;
import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
+import org.apache.iotdb.db.metadata.template.Template;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.sys.*;
+import org.apache.iotdb.db.qp.physical.sys.MNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
import org.apache.iotdb.db.query.context.QueryContext;
import org.apache.iotdb.db.query.control.QueryResourceManager;
import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
@@ -41,7 +53,9 @@
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.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
import com.google.gson.Gson;
import com.google.gson.GsonBuilder;
@@ -53,14 +67,31 @@
import java.io.File;
import java.io.IOException;
import java.io.Serializable;
-import java.util.*;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
import java.util.Map.Entry;
+import java.util.Queue;
+import java.util.Set;
+import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.regex.Pattern;
import java.util.stream.Stream;
import static java.util.stream.Collectors.toList;
-import static org.apache.iotdb.db.conf.IoTDBConstant.*;
+import static org.apache.iotdb.db.conf.IoTDBConstant.LOSS;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_WILDCARD;
+import static org.apache.iotdb.db.conf.IoTDBConstant.SDT;
+import static org.apache.iotdb.db.conf.IoTDBConstant.SDT_COMP_DEV;
+import static org.apache.iotdb.db.conf.IoTDBConstant.SDT_COMP_MAX_TIME;
+import static org.apache.iotdb.db.conf.IoTDBConstant.SDT_COMP_MIN_TIME;
/** The hierarchical struct of the Metadata Tree is implemented in this class. */
public class MTree implements Serializable {
@@ -165,7 +196,7 @@
}
/**
- * Create a timeseries with a full path from root to leaf node Before creating a timeseries, the
+ * Create a timeseries with a full path from root to leaf node. Before creating a timeseries, the
* storage group should be set first, throw exception otherwise
*
* @param path timeseries path
@@ -190,6 +221,7 @@
checkTimeseries(path.getFullPath());
MNode cur = root;
boolean hasSetStorageGroup = false;
+ Template upperTemplate = cur.getDeviceTemplate();
// e.g, path = root.sg.d1.s1, create internal nodes and set cur to d1 node
for (int i = 1; i < nodeNames.length - 1; i++) {
String nodeName = nodeNames[i];
@@ -203,6 +235,15 @@
cur.addChild(nodeName, new MNode(cur, nodeName));
}
cur = cur.getChild(nodeName);
+
+ if (cur.getDeviceTemplate() != null) {
+ upperTemplate = cur.getDeviceTemplate();
+ }
+ }
+
+ if (upperTemplate != null && !upperTemplate.isCompatible(path)) {
+ throw new PathAlreadyExistException(
+ path.getFullPath() + " ( which is incompatible with template )");
}
if (props != null && props.containsKey(LOSS) && props.get(LOSS).equals(SDT)) {
@@ -244,6 +285,89 @@
}
}
+ /**
+ * Create aligned timeseries with full paths from root to one leaf node. Before creating
+ * timeseries, the * storage group should be set first, throw exception otherwise
+ *
+ * @param devicePath device path
+ * @param measurements measurements list
+ * @param dataTypes data types list
+ * @param encodings encodings list
+ * @param compressor compressor
+ */
+ void createAlignedTimeseries(
+ PartialPath devicePath,
+ List<String> measurements,
+ List<TSDataType> dataTypes,
+ List<TSEncoding> encodings,
+ CompressionType compressor)
+ throws MetadataException {
+ String[] deviceNodeNames = devicePath.getNodes();
+ if (deviceNodeNames.length <= 1 || !deviceNodeNames[0].equals(root.getName())) {
+ throw new IllegalPathException(devicePath.getFullPath());
+ }
+ checkTimeseries(devicePath.getFullPath());
+ for (String measurement : measurements) {
+ checkTimeseries(measurement);
+ }
+ MNode cur = root;
+ boolean hasSetStorageGroup = false;
+ StorageGroupMNode storageGroupMNode = null;
+ // e.g, devicePath = root.sg.d1, create internal nodes and set cur to d1 node
+ for (int i = 1; i < deviceNodeNames.length; i++) {
+ String nodeName = deviceNodeNames[i];
+ if (cur instanceof StorageGroupMNode) {
+ hasSetStorageGroup = true;
+ storageGroupMNode = (StorageGroupMNode) cur;
+ }
+ if (!cur.hasChild(nodeName)) {
+ if (!hasSetStorageGroup) {
+ throw new StorageGroupNotSetException("Storage group should be created first");
+ }
+ cur.addChild(nodeName, new MNode(cur, nodeName));
+ }
+ cur = cur.getChild(nodeName);
+ }
+ int alignedTimeseriesIndex = 0;
+ if (storageGroupMNode != null) {
+ alignedTimeseriesIndex = storageGroupMNode.getAlignedTimeseriesIndex();
+ storageGroupMNode.addAlignedTimeseriesIndex();
+ }
+ String leafName = IoTDBConstant.ALIGN_TIMESERIES_PREFIX + alignedTimeseriesIndex;
+
+ // synchronize check and add, we need addChild and add Alias become atomic operation
+ // only write on mtree will be synchronized
+ synchronized (this) {
+ MNode child = cur.getChild(leafName);
+ if (child instanceof MeasurementMNode || child instanceof StorageGroupMNode) {
+ throw new PathAlreadyExistException(devicePath.getFullPath() + "." + leafName);
+ }
+
+ int measurementsSize = measurements.size();
+
+ // this measurementMNode could be a leaf or not.
+ MeasurementMNode measurementMNode =
+ new MeasurementMNode(
+ cur,
+ leafName,
+ new VectorMeasurementSchema(
+ leafName,
+ measurements.toArray(new String[measurementsSize]),
+ dataTypes.toArray(new TSDataType[measurementsSize]),
+ encodings.toArray(new TSEncoding[measurementsSize]),
+ compressor),
+ null);
+ cur.addChild(leafName, measurementMNode);
+ for (String measurement : measurements) {
+ if (child != null) {
+ cur.replaceChild(measurementMNode.getName(), measurementMNode);
+ } else {
+ cur.addChild(measurement, measurementMNode);
+ }
+ }
+ }
+ }
+
private void checkTimeseries(String timeseries) throws IllegalPathException {
if (!IoTDBConfig.NODE_PATTERN.matcher(timeseries).matches()) {
throw new IllegalPathException(
@@ -304,12 +428,14 @@
*
* <p>e.g., get root.sg.d1, get or create all internal nodes and return the node of d1
*/
- MNode getDeviceNodeWithAutoCreating(PartialPath deviceId, int sgLevel) throws MetadataException {
+ Pair<MNode, Template> getDeviceNodeWithAutoCreating(PartialPath deviceId, int sgLevel)
+ throws MetadataException {
String[] nodeNames = deviceId.getNodes();
if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
throw new IllegalPathException(deviceId.getFullPath());
}
MNode cur = root;
+ Template upperTemplate = null;
for (int i = 1; i < nodeNames.length; i++) {
if (!cur.hasChild(nodeNames[i])) {
if (i == sgLevel) {
@@ -321,9 +447,12 @@
cur.addChild(nodeNames[i], new MNode(cur, nodeNames[i]));
}
}
+ // update upper template
+ upperTemplate = cur.getDeviceTemplate() == null ? upperTemplate : cur.getDeviceTemplate();
cur = cur.getChild(nodeNames[i]);
}
- return cur;
+
+ return new Pair<>(cur, upperTemplate);
}
/**
@@ -472,9 +601,11 @@
if (nodes.length == 0 || !IoTDBConstant.PATH_ROOT.equals(nodes[0])) {
throw new IllegalPathException(path.getFullPath());
}
- // delete the last node of path
- curNode.getParent().deleteChild(curNode.getName());
+
MeasurementMNode deletedNode = (MeasurementMNode) curNode;
+
+ // delete the last node of path
+ curNode.getParent().deleteChild(path.getMeasurement());
if (deletedNode.getAlias() != null) {
curNode.getParent().deleteAliasChild(((MeasurementMNode) curNode).getAlias());
}
@@ -496,7 +627,7 @@
/**
* Get measurement schema for a given path. Path must be a complete Path from root to leaf node.
*/
- MeasurementSchema getSchema(PartialPath path) throws MetadataException {
+ IMeasurementSchema getSchema(PartialPath path) throws MetadataException {
MeasurementMNode node = (MeasurementMNode) getNodeByPath(path);
return node.getSchema();
}
@@ -505,7 +636,8 @@
* Get node by path with storage group check If storage group is not set,
* StorageGroupNotSetException will be thrown
*/
- MNode getNodeByPathWithStorageGroupCheck(PartialPath path) throws MetadataException {
+ Pair<MNode, Template> getNodeByPathWithStorageGroupCheck(PartialPath path)
+ throws MetadataException {
boolean storageGroupChecked = false;
String[] nodes = path.getNodes();
if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
@@ -513,7 +645,10 @@
}
MNode cur = root;
+ Template upperTemplate = null;
+
for (int i = 1; i < nodes.length; i++) {
+ upperTemplate = cur.getDeviceTemplate() == null ? upperTemplate : cur.getDeviceTemplate();
cur = cur.getChild(nodes[i]);
if (cur == null) {
// not find
@@ -531,7 +666,7 @@
if (!storageGroupChecked) {
throw new StorageGroupNotSetException(path.getFullPath());
}
- return cur;
+ return new Pair<>(cur, upperTemplate);
}
/**
@@ -580,11 +715,31 @@
throw new IllegalPathException(path.getFullPath());
}
MNode cur = root;
+ Template upperTemplate = cur.getDeviceTemplate();
+
for (int i = 1; i < nodes.length; i++) {
- cur = cur.getChild(nodes[i]);
- if (cur == null) {
- throw new PathNotExistException(path.getFullPath(), true);
+ if (cur.getDeviceTemplate() != null) {
+ upperTemplate = cur.getDeviceTemplate();
}
+ MNode next = cur.getChild(nodes[i]);
+ if (next == null) {
+ if (upperTemplate == null) {
+ throw new PathNotExistException(path.getFullPath(), true);
+ }
+
+ String realName = nodes[i];
+ if (path instanceof VectorPartialPath) {
+ VectorPartialPath vectorPartialPath = (VectorPartialPath) path;
+ realName = vectorPartialPath.getSubSensorsPathList().get(0).getMeasurement();
+ }
+
+ IMeasurementSchema schema = upperTemplate.getSchemaMap().get(realName);
+ if (schema == null) {
+ throw new PathNotExistException(path.getFullPath(), true);
+ }
+ return new MeasurementMNode(cur, schema.getMeasurementId(), schema, null);
+ }
+ cur = next;
}
return cur;
}
@@ -1004,7 +1159,7 @@
}
List<Pair<PartialPath, String[]>> allMatchedNodes = new ArrayList<>();
- findPath(root, nodes, 1, allMatchedNodes, false, true, queryContext);
+ findPath(root, nodes, 1, allMatchedNodes, false, true, queryContext, null);
Stream<Pair<PartialPath, String[]>> sortedStream =
allMatchedNodes.stream()
@@ -1043,7 +1198,7 @@
offset.set(plan.getOffset());
curOffset.set(-1);
count.set(0);
- findPath(root, nodes, 1, res, offset.get() != 0 || limit.get() != 0, false, null);
+ findPath(root, nodes, 1, res, offset.get() != 0 || limit.get() != 0, false, null, null);
// avoid memory leaks
limit.remove();
offset.remove();
@@ -1069,7 +1224,8 @@
List<Pair<PartialPath, String[]>> timeseriesSchemaList,
boolean hasLimit,
boolean needLast,
- QueryContext queryContext)
+ QueryContext queryContext,
+ Template upperTemplate)
throws MetadataException {
if (node instanceof MeasurementMNode && nodes.length <= idx) {
if (hasLimit) {
@@ -1078,11 +1234,127 @@
return;
}
}
+ IMeasurementSchema measurementSchema = ((MeasurementMNode) node).getSchema();
+ if (measurementSchema instanceof MeasurementSchema) {
+ addMeasurementSchema(
+ node, timeseriesSchemaList, needLast, queryContext, measurementSchema, "*");
+ } else if (measurementSchema instanceof VectorMeasurementSchema) {
+ String lastWord = nodes[nodes.length - 1];
+ addVectorMeasurementSchema(
+ node,
+ timeseriesSchemaList,
+ needLast,
+ queryContext,
+ measurementSchema,
+ nodes.length == idx ? lastWord : "*");
+ }
+ if (hasLimit) {
+ count.set(count.get() + 1);
+ }
+ }
+ String nodeReg = MetaUtils.getNodeRegByIdx(idx, nodes);
+ if (node.getDeviceTemplate() != null) {
+ upperTemplate = node.getDeviceTemplate();
+ }
+
+ // we should use template when all child is measurement or this node has no child
+ if (!nodeReg.contains(PATH_WILDCARD)) {
+ MNode next = null;
+ if (nodeReg.contains("(") && nodeReg.contains(",")) {
+ next = node.getChildOfAlignedTimeseries(nodeReg);
+ } else {
+ next = node.getChild(nodeReg);
+ }
+ if (next != null) {
+ findPath(
+ next,
+ nodes,
+ idx + 1,
+ timeseriesSchemaList,
+ hasLimit,
+ needLast,
+ queryContext,
+ upperTemplate);
+ }
+ } else {
+ for (MNode child : node.getDistinctMNodes()) {
+ boolean continueSearch = false;
+ if (child instanceof MeasurementMNode
+ && ((MeasurementMNode) child).getSchema() instanceof VectorMeasurementSchema) {
+ List<String> measurementsList =
+ ((MeasurementMNode) child).getSchema().getValueMeasurementIdList();
+ for (String measurement : measurementsList) {
+ if (Pattern.matches(nodeReg.replace("*", ".*"), measurement)) {
+ continueSearch = true;
+ }
+ }
+ } else {
+ if (Pattern.matches(nodeReg.replace("*", ".*"), child.getName())) {
+ continueSearch = true;
+ }
+ }
+ if (!continueSearch) {
+ continue;
+ }
+
+ findPath(
+ child,
+ nodes,
+ idx + 1,
+ timeseriesSchemaList,
+ hasLimit,
+ needLast,
+ queryContext,
+ upperTemplate);
+ if (hasLimit && count.get().intValue() == limit.get().intValue()) {
+ break;
+ }
+ }
+ }
+
+ // template part
+ if (!(node instanceof MeasurementMNode) && node.isUseTemplate()) {
+ if (upperTemplate != null) {
+ HashSet<IMeasurementSchema> set = new HashSet<>();
+ for (IMeasurementSchema schema : upperTemplate.getSchemaMap().values()) {
+ if (set.add(schema)) {
+ if (schema instanceof MeasurementSchema) {
+ addMeasurementSchema(
+ new MeasurementMNode(node, schema.getMeasurementId(), schema, null),
+ timeseriesSchemaList,
+ needLast,
+ queryContext,
+ schema,
+ nodeReg);
+ } else if (schema instanceof VectorMeasurementSchema) {
+ String firstNode = schema.getValueMeasurementIdList().get(0);
+ addVectorMeasurementSchema(
+ new MeasurementMNode(node, firstNode, schema, null),
+ timeseriesSchemaList,
+ needLast,
+ queryContext,
+ schema,
+ nodeReg);
+ }
+ }
+ }
+ }
+ }
+ }
+
+ private void addMeasurementSchema(
+ MNode node,
+ List<Pair<PartialPath, String[]>> timeseriesSchemaList,
+ boolean needLast,
+ QueryContext queryContext,
+ IMeasurementSchema measurementSchema,
+ String reg)
+ throws StorageGroupNotSetException {
+ if (Pattern.matches(reg.replace("*", ".*"), measurementSchema.getMeasurementId())) {
PartialPath nodePath = node.getPartialPath();
String[] tsRow = new String[7];
tsRow[0] = ((MeasurementMNode) node).getAlias();
- MeasurementSchema measurementSchema = ((MeasurementMNode) node).getSchema();
tsRow[1] = getStorageGroupPath(nodePath).getFullPath();
tsRow[2] = measurementSchema.getType().toString();
tsRow[3] = measurementSchema.getEncodingType().toString();
@@ -1092,27 +1364,44 @@
needLast ? String.valueOf(getLastTimeStamp((MeasurementMNode) node, queryContext)) : null;
Pair<PartialPath, String[]> temp = new Pair<>(nodePath, tsRow);
timeseriesSchemaList.add(temp);
-
- if (hasLimit) {
- count.set(count.get() + 1);
- }
}
- String nodeReg = MetaUtils.getNodeRegByIdx(idx, nodes);
- if (!nodeReg.contains(PATH_WILDCARD)) {
- MNode next = node.getChild(nodeReg);
- if (next != null) {
- findPath(next, nodes, idx + 1, timeseriesSchemaList, hasLimit, needLast, queryContext);
+ }
+
+ private void addVectorMeasurementSchema(
+ MNode node,
+ List<Pair<PartialPath, String[]>> timeseriesSchemaList,
+ boolean needLast,
+ QueryContext queryContext,
+ IMeasurementSchema schema,
+ String reg)
+ throws StorageGroupNotSetException, IllegalPathException {
+ List<String> measurements = schema.getValueMeasurementIdList();
+ int measurementSize = measurements.size();
+ Set<String> measurementsInReg = new HashSet<>();
+ if (reg.contains("(") && reg.contains(",")) {
+ measurementsInReg.addAll(MetaUtils.getMeasurementsInPartialPath(reg));
+ }
+ for (int i = 0; i < measurementSize; i++) {
+ if (measurementsInReg.size() != 0 && !measurementsInReg.contains(measurements.get(i))) {
+ continue;
}
- } else {
- for (MNode child : node.getChildren().values()) {
- if (!Pattern.matches(nodeReg.replace("*", ".*"), child.getName())) {
- continue;
- }
- findPath(child, nodes, idx + 1, timeseriesSchemaList, hasLimit, needLast, queryContext);
- if (hasLimit && count.get().intValue() == limit.get().intValue()) {
- return;
- }
+ if (measurementsInReg.size() == 0
+ && !Pattern.matches(reg.replace("*", ".*"), measurements.get(i))) {
+ continue;
}
+ PartialPath devicePath = node.getPartialPath().getDevicePath();
+ String[] tsRow = new String[7];
+ tsRow[0] = null;
+ tsRow[1] = getStorageGroupPath(devicePath).getFullPath();
+ tsRow[2] = schema.getValueTSDataTypeList().get(i).toString();
+ tsRow[3] = schema.getValueTSEncodingList().get(i).toString();
+ tsRow[4] = schema.getCompressor().toString();
+ tsRow[5] = "-1";
+ tsRow[6] =
+ needLast ? String.valueOf(getLastTimeStamp((MeasurementMNode) node, queryContext)) : null;
+ Pair<PartialPath, String[]> temp =
+ new Pair<>(new PartialPath(devicePath.getFullPath(), measurements.get(i)), tsRow);
+ timeseriesSchemaList.add(temp);
}
}
@@ -1193,7 +1482,7 @@
* <p>e.g., MTree has [root.sg1.d1.s1, root.sg1.d1.s2, root.sg1.d2.s1] given path = root.sg1.d1
* return [s1, s2]
*
- * @param partial Path
+ * @param path Path
* @return All child nodes' seriesPath(s) of given seriesPath.
*/
Set<String> getChildNodeInNextLevel(PartialPath path) throws MetadataException {
@@ -1266,7 +1555,7 @@
throw new IllegalPathException(prefixPath.getFullPath());
}
Set<PartialPath> devices = new TreeSet<>();
- findDevices(root, nodes, 1, devices, false);
+ findDevices(root, nodes, 1, devices, false, null);
return devices;
}
@@ -1280,7 +1569,7 @@
offset.set(plan.getOffset());
curOffset.set(-1);
count.set(0);
- findDevices(root, nodes, 1, devices, offset.get() != 0 || limit.get() != 0);
+ findDevices(root, nodes, 1, devices, offset.get() != 0 || limit.get() != 0, null);
// avoid memory leaks
limit.remove();
offset.remove();
@@ -1308,7 +1597,13 @@
*/
@SuppressWarnings("squid:S3776")
private void findDevices(
- MNode node, String[] nodes, int idx, Set<PartialPath> res, boolean hasLimit) {
+ MNode node,
+ String[] nodes,
+ int idx,
+ Set<PartialPath> res,
+ boolean hasLimit,
+ Template upperTemplate) {
+ upperTemplate = node.getDeviceTemplate() == null ? upperTemplate : node.getDeviceTemplate();
String nodeReg = MetaUtils.getNodeRegByIdx(idx, nodes);
// the node path doesn't contains '*'
if (!nodeReg.contains(PATH_WILDCARD)) {
@@ -1325,12 +1620,18 @@
}
res.add(node.getPartialPath());
} else {
- findDevices(next, nodes, idx + 1, res, hasLimit);
+ findDevices(next, nodes, idx + 1, res, hasLimit, upperTemplate);
}
}
} else { // the node path contains '*'
boolean deviceAdded = false;
- for (MNode child : node.getChildren().values()) {
+ List<MNode> children = new ArrayList<>(node.getChildren().values());
+ // template part
+ if (upperTemplate != null && node.isUseTemplate()) {
+ children.addAll(upperTemplate.getMeasurementMNode());
+ }
+
+ for (MNode child : children) {
// use '.*' to replace '*' to form a regex to match
// if the match failed, skip it.
if (!Pattern.matches(nodeReg.replace("*", ".*"), child.getName())) {
@@ -1348,7 +1649,7 @@
res.add(node.getPartialPath());
deviceAdded = true;
}
- findDevices(child, nodes, idx + 1, res, hasLimit);
+ findDevices(child, nodes, idx + 1, res, hasLimit, upperTemplate);
}
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MeasurementMeta.java b/server/src/main/java/org/apache/iotdb/db/metadata/MeasurementMeta.java
index 7963c7b..59a5068 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MeasurementMeta.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MeasurementMeta.java
@@ -19,34 +19,34 @@
package org.apache.iotdb.db.metadata;
import org.apache.iotdb.tsfile.read.TimeValuePair;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
public class MeasurementMeta {
- private MeasurementSchema measurementSchema = null;
+ private IMeasurementSchema measurementSchema;
private String alias = null; // TODO get schema by alias
private TimeValuePair timeValuePair = null;
public MeasurementMeta(
- MeasurementSchema measurementSchema, String alias, TimeValuePair timeValuePair) {
+ IMeasurementSchema measurementSchema, String alias, TimeValuePair timeValuePair) {
this.measurementSchema = measurementSchema;
this.alias = alias;
this.timeValuePair = timeValuePair;
}
- public MeasurementMeta(MeasurementSchema measurementSchema, String alias) {
+ public MeasurementMeta(IMeasurementSchema measurementSchema, String alias) {
this.measurementSchema = measurementSchema;
this.alias = alias;
}
- public MeasurementMeta(MeasurementSchema measurementSchema) {
+ public MeasurementMeta(IMeasurementSchema measurementSchema) {
this.measurementSchema = measurementSchema;
}
- public MeasurementSchema getMeasurementSchema() {
+ public IMeasurementSchema getMeasurementSchema() {
return measurementSchema;
}
- public void setMeasurementSchema(MeasurementSchema measurementSchema) {
+ public void setMeasurementSchema(IMeasurementSchema measurementSchema) {
this.measurementSchema = measurementSchema;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MetaUtils.java b/server/src/main/java/org/apache/iotdb/db/metadata/MetaUtils.java
index 7b9fe11..3d29697 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MetaUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MetaUtils.java
@@ -25,6 +25,7 @@
import org.apache.iotdb.db.utils.TestOnly;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
@@ -106,6 +107,29 @@
return new PartialPath(storageGroupNodes);
}
+ /**
+ * get aligned measurements in partial path FIXME maybe called by prefix path
+ *
+ * @param fullPath partial. For example: root.sg1.d1.(s1, s2, s3)
+ * @return measurement names. For example: [s1, s2, s3]
+ */
+ public static List<String> getMeasurementsInPartialPath(PartialPath fullPath) {
+ if (fullPath.getMeasurement().contains("(") && fullPath.getMeasurement().contains(",")) {
+ return getMeasurementsInPartialPath(fullPath.getMeasurement());
+ } else {
+ return Arrays.asList(fullPath.getMeasurement());
+ }
+ }
+
+ public static List<String> getMeasurementsInPartialPath(String measurementString) {
+ String[] measurements = measurementString.replace("(", "").replace(")", "").split(",");
+ List<String> measurementList = new ArrayList<>();
+ for (String measurement : measurements) {
+ measurementList.add(measurement.trim());
+ }
+ return measurementList;
+ }
+
@TestOnly
public static List<String> getMultiFullPaths(MNode node) {
if (node == null) {
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MetadataOperationType.java b/server/src/main/java/org/apache/iotdb/db/metadata/MetadataOperationType.java
index 466c0a2..deb3272 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MetadataOperationType.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MetadataOperationType.java
@@ -27,6 +27,8 @@
public static final String CREATE_TIMESERIES = "0";
public static final String DELETE_TIMESERIES = "1";
public static final String SET_STORAGE_GROUP = "2";
+ public static final String CREATE_ALIGNED_TIMESERIES = "3";
+ public static final String AUTO_CREATE_DEVICE_MNODE = "4";
public static final String SET_TTL = "10";
public static final String DELETE_STORAGE_GROUP = "11";
public static final String CREATE_INDEX = "31";
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java b/server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java
index 1f006d9..79d075f 100755
--- a/server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java
@@ -40,11 +40,11 @@
private static final Logger logger = LoggerFactory.getLogger(PartialPath.class);
- private String[] nodes;
+ protected String[] nodes;
// alias of measurement, null pointer cannot be serialized in thrift so empty string is instead
- private String measurementAlias = "";
+ protected String measurementAlias = "";
// alias of time series used in SELECT AS
- private String tsAlias = "";
+ protected String tsAlias = "";
/**
* Construct the PartialPath using a String, will split the given String into String[] E.g., path
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/VectorPartialPath.java b/server/src/main/java/org/apache/iotdb/db/metadata/VectorPartialPath.java
new file mode 100644
index 0000000..b4cdcc4
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/VectorPartialPath.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.metadata;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+
+import java.util.List;
+import java.util.Objects;
+
+public class VectorPartialPath extends PartialPath {
+
+ private List<PartialPath> subSensorsPathList;
+
+ public VectorPartialPath(String path, List<PartialPath> subSensorsPathList)
+ throws IllegalPathException {
+ super(path);
+ this.subSensorsPathList = subSensorsPathList;
+ }
+
+ public List<PartialPath> getSubSensorsPathList() {
+ return subSensorsPathList;
+ }
+
+ public void setSubSensorsPathList(List<PartialPath> subSensorsPathList) {
+ this.subSensorsPathList = subSensorsPathList;
+ }
+
+ public void addSubSensor(PartialPath path) {
+ this.subSensorsPathList.add(path);
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ if (!super.equals(o)) {
+ return false;
+ }
+ VectorPartialPath that = (VectorPartialPath) o;
+ return Objects.equals(subSensorsPathList, that.subSensorsPathList);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(super.hashCode(), subSensorsPathList);
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java
index 8fda60d..afde6dc 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/logfile/MLogWriter.java
@@ -28,8 +28,12 @@
import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.db.qp.physical.crud.SetDeviceTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.AutoCreateDeviceMNodePlan;
import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+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.DeleteStorageGroupPlan;
import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
@@ -37,6 +41,7 @@
import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetUsingDeviceTemplatePlan;
import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
import org.apache.iotdb.db.writelog.io.LogWriter;
import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
@@ -121,6 +126,11 @@
putLog(createTimeSeriesPlan);
}
+ public void createAlignedTimeseries(CreateAlignedTimeSeriesPlan createAlignedTimeSeriesPlan)
+ throws IOException {
+ putLog(createAlignedTimeSeriesPlan);
+ }
+
public void deleteTimeseries(DeleteTimeSeriesPlan deleteTimeSeriesPlan) throws IOException {
putLog(deleteTimeSeriesPlan);
}
@@ -151,6 +161,18 @@
putLog(plan);
}
+ public void createDeviceTemplate(CreateTemplatePlan plan) throws IOException {
+ putLog(plan);
+ }
+
+ public void setDeviceTemplate(SetDeviceTemplatePlan plan) throws IOException {
+ putLog(plan);
+ }
+
+ public void autoCreateDeviceMNode(AutoCreateDeviceMNodePlan plan) throws IOException {
+ putLog(plan);
+ }
+
public void serializeMNode(MNode node) throws IOException {
int childSize = 0;
if (node.getChildren() != null) {
@@ -177,7 +199,13 @@
childSize = node.getChildren().size();
}
StorageGroupMNodePlan plan =
- new StorageGroupMNodePlan(node.getName(), node.getDataTTL(), childSize);
+ new StorageGroupMNodePlan(
+ node.getName(), node.getDataTTL(), childSize, node.getAlignedTimeseriesIndex());
+ putLog(plan);
+ }
+
+ public void setUsingDeviceTemplate(PartialPath path) throws IOException {
+ SetUsingDeviceTemplatePlan plan = new SetUsingDeviceTemplatePlan(path);
putLog(plan);
}
@@ -348,6 +376,9 @@
plan.setTagOffset(offset);
createTimeseries(plan);
break;
+ case MetadataOperationType.CREATE_ALIGNED_TIMESERIES:
+ case MetadataOperationType.AUTO_CREATE_DEVICE_MNODE:
+ throw new MetadataException("Impossible operation!");
case MetadataOperationType.DELETE_TIMESERIES:
if (args.length > 2) {
StringBuilder tmp = new StringBuilder();
@@ -407,7 +438,10 @@
CompressionType.values()[Integer.parseInt(words[5])]));
case "1":
return new StorageGroupMNodePlan(
- words[1], Long.parseLong(words[2]), Integer.parseInt(words[3]));
+ words[1],
+ Long.parseLong(words[2]),
+ Integer.parseInt(words[3]),
+ words.length == 5 ? Integer.parseInt(words[4]) : 0);
case "0":
return new MNodePlan(words[1], Integer.parseInt(words[2]));
default:
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MNode.java
index e5f03ae..f465952 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MNode.java
@@ -19,8 +19,12 @@
package org.apache.iotdb.db.metadata.mnode;
import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MetaUtils;
import org.apache.iotdb.db.metadata.PartialPath;
import org.apache.iotdb.db.metadata.logfile.MLogWriter;
+import org.apache.iotdb.db.metadata.template.Template;
import org.apache.iotdb.db.rescon.CachedStringPool;
import java.io.IOException;
@@ -30,6 +34,7 @@
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
+import java.util.Objects;
import java.util.concurrent.ConcurrentHashMap;
/**
@@ -67,6 +72,11 @@
@SuppressWarnings("squid:S3077")
private transient volatile Map<String, MNode> aliasChildren = null;
+ // device template
+ protected Template deviceTemplate = null;
+
+ private volatile boolean useTemplate = false;
+
/** Constructor of MNode. */
public MNode(MNode parent, String name) {
this.parent = parent;
@@ -147,6 +157,14 @@
}
}
+ public Template getDeviceTemplate() {
+ return deviceTemplate;
+ }
+
+ public void setDeviceTemplate(Template deviceTemplate) {
+ this.deviceTemplate = deviceTemplate;
+ }
+
/** get the child with the name */
public MNode getChild(String name) {
MNode child = null;
@@ -159,6 +177,24 @@
return aliasChildren == null ? null : aliasChildren.get(name);
}
+ public MNode getChildOfAlignedTimeseries(String name) throws MetadataException {
+ MNode node = null;
+ // for aligned timeseries
+ List<String> measurementList = MetaUtils.getMeasurementsInPartialPath(name);
+ for (String measurement : measurementList) {
+ MNode nodeOfMeasurement = getChild(measurement);
+ if (node == null) {
+ node = nodeOfMeasurement;
+ } else {
+ if (node != nodeOfMeasurement) {
+ throw new AlignedTimeseriesException(
+ "Cannot get node of children in different aligned timeseries", name);
+ }
+ }
+ }
+ return node;
+ }
+
/** get the count of all MeasurementMNode whose ancestor is current node */
public int getMeasurementMNodeCount() {
if (children == null) {
@@ -243,6 +279,19 @@
return children;
}
+ public List<MNode> getDistinctMNodes() {
+ if (children == null) {
+ return Collections.emptyList();
+ }
+ List<MNode> distinctList = new ArrayList<>();
+ for (MNode child : children.values()) {
+ if (!distinctList.contains(child)) {
+ distinctList.add(child);
+ }
+ }
+ return distinctList;
+ }
+
public Map<String, MNode> getAliasChildren() {
if (aliasChildren == null) {
return Collections.emptyMap();
@@ -303,4 +352,49 @@
this.deleteChild(measurement);
this.addChild(newChildNode.getName(), newChildNode);
}
+
+ public Template getUpperTemplate() {
+ MNode cur = this;
+ while (cur != null) {
+ if (cur.getDeviceTemplate() != null) {
+ return cur.deviceTemplate;
+ }
+ cur = cur.parent;
+ }
+
+ return null;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ MNode mNode = (MNode) o;
+ if (fullPath == null) {
+ return Objects.equals(getFullPath(), mNode.getFullPath());
+ } else {
+ return Objects.equals(fullPath, mNode.fullPath);
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ if (fullPath == null) {
+ return Objects.hash(getFullPath());
+ } else {
+ return Objects.hash(fullPath);
+ }
+ }
+
+ public boolean isUseTemplate() {
+ return useTemplate;
+ }
+
+ public void setUseTemplate(boolean useTemplate) {
+ this.useTemplate = useTemplate;
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
index ad898d5..fb24c04 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java
@@ -25,6 +25,7 @@
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.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import java.io.IOException;
@@ -37,7 +38,7 @@
private static final long serialVersionUID = -1199657856921206435L;
/** measurement's Schema for one timeseries represented by current leaf node */
- private MeasurementSchema schema;
+ private IMeasurementSchema schema;
private String alias;
// tag/attribute's start offset in tag file
@@ -62,13 +63,13 @@
}
public MeasurementMNode(
- MNode parent, String measurementName, MeasurementSchema schema, String alias) {
+ MNode parent, String measurementName, IMeasurementSchema schema, String alias) {
super(parent, measurementName);
this.schema = schema;
this.alias = alias;
}
- public MeasurementSchema getSchema() {
+ public IMeasurementSchema getSchema() {
return schema;
}
@@ -126,7 +127,7 @@
this.alias = alias;
}
- public void setSchema(MeasurementSchema schema) {
+ public void setSchema(IMeasurementSchema schema) {
this.schema = schema;
}
@@ -158,7 +159,7 @@
props.put(propInfo.split(":")[0], propInfo.split(":")[1]);
}
}
- MeasurementSchema schema =
+ IMeasurementSchema schema =
new MeasurementSchema(
name,
Byte.parseByte(nodeInfo[3]),
@@ -178,4 +179,13 @@
return node;
}
+
+ public TSDataType getDataType(String measurementId) {
+ if (schema instanceof MeasurementSchema) {
+ return schema.getType();
+ } else {
+ int index = schema.getMeasurementIdColumnIndex(measurementId);
+ return schema.getValueTSDataTypeList().get(index);
+ }
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/StorageGroupMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/StorageGroupMNode.java
index 016c0b2..cfb2a71 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/StorageGroupMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/StorageGroupMNode.java
@@ -33,9 +33,18 @@
*/
private long dataTTL;
+ private int alignedTimeseriesIndex;
+
public StorageGroupMNode(MNode parent, String name, long dataTTL) {
super(parent, name);
this.dataTTL = dataTTL;
+ this.alignedTimeseriesIndex = 0;
+ }
+
+ public StorageGroupMNode(MNode parent, String name, long dataTTL, int alignedTimeseriesIndex) {
+ super(parent, name);
+ this.dataTTL = dataTTL;
+ this.alignedTimeseriesIndex = alignedTimeseriesIndex;
}
public long getDataTTL() {
@@ -46,6 +55,14 @@
this.dataTTL = dataTTL;
}
+ public int getAlignedTimeseriesIndex() {
+ return alignedTimeseriesIndex;
+ }
+
+ public void addAlignedTimeseriesIndex() {
+ this.alignedTimeseriesIndex++;
+ }
+
@Override
public void serializeTo(MLogWriter logWriter) throws IOException {
serializeChildren(logWriter);
@@ -54,10 +71,15 @@
}
public static StorageGroupMNode deserializeFrom(StorageGroupMNodePlan plan) {
- return new StorageGroupMNode(null, plan.getName(), plan.getDataTTL());
+ return new StorageGroupMNode(
+ null, plan.getName(), plan.getDataTTL(), plan.getAlignedTimeseriesIndex());
}
public static StorageGroupMNode deserializeFrom(String[] nodeInfo) {
- return new StorageGroupMNode(null, nodeInfo[1], Long.valueOf(nodeInfo[2]));
+ return new StorageGroupMNode(
+ null,
+ nodeInfo[1],
+ Long.parseLong(nodeInfo[2]),
+ nodeInfo.length == 4 ? Integer.parseInt(nodeInfo[3]) : 0);
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java b/server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java
new file mode 100644
index 0000000..84bf240
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java
@@ -0,0 +1,170 @@
+/*
+ * 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.metadata.template;
+
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
+
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class Template {
+ private static final AtomicLong increasingId = new AtomicLong();
+
+ String name;
+
+ Map<String, IMeasurementSchema> schemaMap = new HashMap<>();
+
+ public Template(CreateTemplatePlan plan) {
+ name = plan.getName();
+
+ // put measurement into a map
+ for (int i = 0; i < plan.getMeasurements().size(); i++) {
+ IMeasurementSchema curSchema;
+ // vector
+ int size = plan.getMeasurements().get(i).size();
+ if (size > 1) {
+ String[] measurementsArray = new String[size];
+ TSDataType[] typeArray = new TSDataType[size];
+ TSEncoding[] encodingArray = new TSEncoding[size];
+
+ for (int j = 0; j < size; j++) {
+ measurementsArray[j] = plan.getMeasurements().get(i).get(j);
+ typeArray[j] = plan.getDataTypes().get(i).get(j);
+ encodingArray[j] = plan.getEncodings().get(i).get(j);
+ }
+
+ curSchema =
+ new VectorMeasurementSchema(
+ IoTDBConstant.ALIGN_TIMESERIES_PREFIX + "#" + increasingId.getAndIncrement(),
+ measurementsArray,
+ typeArray,
+ encodingArray,
+ plan.getCompressors().get(i));
+ }
+ // normal measurement
+ else {
+ curSchema =
+ new MeasurementSchema(
+ plan.getMeasurements().get(i).get(0),
+ plan.getDataTypes().get(i).get(0),
+ plan.getEncodings().get(i).get(0),
+ plan.getCompressors().get(i));
+ }
+
+ for (String path : plan.getMeasurements().get(i)) {
+ if (schemaMap.containsKey(path)) {
+ throw new IllegalArgumentException(
+ "Duplicate measurement name in create template plan. Name is :" + path);
+ }
+
+ schemaMap.put(path, curSchema);
+ }
+ }
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public void setName(String name) {
+ this.name = name;
+ }
+
+ public Map<String, IMeasurementSchema> getSchemaMap() {
+ return schemaMap;
+ }
+
+ public void setSchemaMap(Map<String, IMeasurementSchema> schemaMap) {
+ this.schemaMap = schemaMap;
+ }
+
+ public boolean isCompatible(PartialPath path) {
+ return !schemaMap.containsKey(path.getMeasurement());
+ }
+
+ @TestOnly
+ public static void clear() {
+ increasingId.set(0);
+ }
+
+ public List<MeasurementMNode> getMeasurementMNode() {
+ Set<IMeasurementSchema> deduplicateSchema = new HashSet<>();
+ List<MeasurementMNode> res = new ArrayList<>();
+
+ for (IMeasurementSchema measurementSchema : schemaMap.values()) {
+ if (deduplicateSchema.add(measurementSchema)) {
+ MeasurementMNode measurementMNode = null;
+ if (measurementSchema instanceof MeasurementSchema) {
+ measurementMNode =
+ new MeasurementMNode(
+ null, measurementSchema.getMeasurementId(), measurementSchema, null);
+
+ } else if (measurementSchema instanceof VectorMeasurementSchema) {
+ measurementMNode =
+ new MeasurementMNode(
+ null,
+ getMeasurementNodeName(measurementSchema.getValueMeasurementIdList().get(0)),
+ measurementSchema,
+ null);
+ }
+
+ res.add(measurementMNode);
+ }
+ }
+
+ return res;
+ }
+
+ public String getMeasurementNodeName(String measurementName) {
+ return schemaMap.get(measurementName).getMeasurementId();
+ }
+
+ @Override
+ public boolean equals(Object t) {
+ if (this == t) {
+ return true;
+ }
+ if (t == null || getClass() != t.getClass()) {
+ return false;
+ }
+ Template that = (Template) t;
+ return this.name.equals(that.name) && this.schemaMap.equals(that.schemaMap);
+ }
+
+ @Override
+ public int hashCode() {
+ return new HashCodeBuilder(17, 37).append(name).append(schemaMap).toHashCode();
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
index 42fc46e..527644f 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
@@ -47,6 +47,7 @@
import org.apache.iotdb.db.exception.metadata.PathNotExistException;
import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.MetaUtils;
import org.apache.iotdb.db.metadata.PartialPath;
import org.apache.iotdb.db.metadata.mnode.MNode;
import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
@@ -59,6 +60,7 @@
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
import org.apache.iotdb.db.qp.physical.crud.AggregationPlan;
import org.apache.iotdb.db.qp.physical.crud.AlignByDevicePlan;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
import org.apache.iotdb.db.qp.physical.crud.DeletePartitionPlan;
import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
import org.apache.iotdb.db.qp.physical.crud.FillQueryPlan;
@@ -74,10 +76,12 @@
import org.apache.iotdb.db.qp.physical.crud.QueryIndexPlan;
import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
import org.apache.iotdb.db.qp.physical.crud.RawDataQueryPlan;
+import org.apache.iotdb.db.qp.physical.crud.SetDeviceTemplatePlan;
import org.apache.iotdb.db.qp.physical.crud.UDTFPlan;
import org.apache.iotdb.db.qp.physical.sys.AlterTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
import org.apache.iotdb.db.qp.physical.sys.CountPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateFunctionPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateMultiTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
@@ -122,14 +126,17 @@
import org.apache.iotdb.db.tools.TsFileRewriteTool;
import org.apache.iotdb.db.utils.AuthUtils;
import org.apache.iotdb.db.utils.FileLoaderUtils;
+import org.apache.iotdb.db.utils.TypeInferenceUtils;
import org.apache.iotdb.db.utils.UpgradeUtils;
import org.apache.iotdb.rpc.RpcUtils;
import org.apache.iotdb.rpc.TSStatusCode;
import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetadata;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import org.apache.iotdb.tsfile.read.common.Field;
import org.apache.iotdb.tsfile.read.common.Path;
@@ -138,7 +145,7 @@
import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
import org.apache.iotdb.tsfile.utils.Binary;
import org.apache.iotdb.tsfile.utils.Pair;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
import org.slf4j.Logger;
@@ -264,6 +271,8 @@
return deleteTimeSeries((DeleteTimeSeriesPlan) plan);
case CREATE_TIMESERIES:
return createTimeSeries((CreateTimeSeriesPlan) plan);
+ case CREATE_ALIGNED_TIMESERIES:
+ return createAlignedTimeSeries((CreateAlignedTimeSeriesPlan) plan);
case CREATE_MULTI_TIMESERIES:
return createMultiTimeSeries((CreateMultiTimeSeriesPlan) plan);
case ALTER_TIMESERIES:
@@ -336,12 +345,36 @@
throw new QueryProcessException(e.getMessage());
}
return true;
+ case CREATE_TEMPLATE:
+ return createDeviceTemplate((CreateTemplatePlan) plan);
+ case SET_DEVICE_TEMPLATE:
+ return setDeviceTemplate((SetDeviceTemplatePlan) plan);
default:
throw new UnsupportedOperationException(
String.format("operation %s is not supported", plan.getOperatorType()));
}
}
+ private boolean createDeviceTemplate(CreateTemplatePlan createTemplatePlan)
+ throws QueryProcessException {
+ try {
+ IoTDB.metaManager.createDeviceTemplate(createTemplatePlan);
+ } catch (MetadataException e) {
+ throw new QueryProcessException(e);
+ }
+ return true;
+ }
+
+ private boolean setDeviceTemplate(SetDeviceTemplatePlan setDeviceTemplatePlan)
+ throws QueryProcessException {
+ try {
+ IoTDB.metaManager.setDeviceTemplate(setDeviceTemplatePlan);
+ } catch (MetadataException e) {
+ throw new QueryProcessException(e);
+ }
+ return true;
+ }
+
private boolean operateCreateFunction(CreateFunctionPlan plan) throws UDFRegistrationException {
UDFRegistrationService.getInstance()
.register(plan.getUdfName(), plan.getClassName(), plan.isTemporary(), true);
@@ -621,6 +654,8 @@
case BOOLEAN:
field.setBoolV(((boolean) val));
break;
+ default:
+ throw new UnSupportedDataTypeException("Unsupported data type" + columnType);
}
RowRecord record = new RowRecord(0);
record.addField(field);
@@ -956,7 +991,7 @@
String.format(
"Cannot load file %s because the file has crashed.", file.getAbsolutePath()));
}
- Map<Path, MeasurementSchema> schemaMap = new HashMap<>();
+ Map<Path, IMeasurementSchema> schemaMap = new HashMap<>();
List<ChunkGroupMetadata> chunkGroupMetadataList = new ArrayList<>();
try (TsFileSequenceReader reader = new TsFileSequenceReader(file.getAbsolutePath(), false)) {
@@ -1002,9 +1037,9 @@
@SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
private void createSchemaAutomatically(
List<ChunkGroupMetadata> chunkGroupMetadataList,
- Map<Path, MeasurementSchema> knownSchemas,
+ Map<Path, IMeasurementSchema> knownSchemas,
int sgLevel)
- throws QueryProcessException, MetadataException {
+ throws QueryProcessException, MetadataException, IOException {
if (chunkGroupMetadataList.isEmpty()) {
return;
}
@@ -1013,7 +1048,9 @@
for (ChunkGroupMetadata chunkGroupMetadata : chunkGroupMetadataList) {
String device = chunkGroupMetadata.getDevice();
MNode node =
- IoTDB.metaManager.getDeviceNodeWithAutoCreate(new PartialPath(device), true, sgLevel);
+ IoTDB.metaManager.getDeviceNodeWithAutoCreate(
+ new PartialPath(device), true, true, sgLevel)
+ .left;
for (ChunkMetadata chunkMetadata : chunkGroupMetadata.getChunkMetadataList()) {
PartialPath series =
new PartialPath(
@@ -1022,7 +1059,7 @@
+ chunkMetadata.getMeasurementUid());
if (!registeredSeries.contains(series)) {
registeredSeries.add(series);
- MeasurementSchema schema =
+ IMeasurementSchema schema =
knownSchemas.get(new Path(series.getDevice(), series.getMeasurement()));
if (schema == null) {
throw new MetadataException(
@@ -1104,7 +1141,11 @@
}
protected MNode getSeriesSchemas(InsertPlan insertPlan) throws MetadataException {
- return IoTDB.metaManager.getSeriesSchemasAndReadLockDevice(insertPlan);
+ try {
+ return IoTDB.metaManager.getSeriesSchemasAndReadLockDevice(insertPlan);
+ } catch (IOException e) {
+ throw new MetadataException(e);
+ }
}
private void checkFailedMeasurments(InsertPlan plan)
@@ -1215,6 +1256,15 @@
try {
insertRowPlan.setMeasurementMNodes(
new MeasurementMNode[insertRowPlan.getMeasurements().length]);
+ // When insert data with sql statement, the data types will be null here.
+ // We need to predicted the data types first
+ if (insertRowPlan.getDataTypes()[0] == null) {
+ for (int i = 0; i < insertRowPlan.getDataTypes().length; i++) {
+ insertRowPlan.getDataTypes()[i] =
+ TypeInferenceUtils.getPredictedDataType(
+ insertRowPlan.getValues()[i], insertRowPlan.isNeedInferType());
+ }
+ }
// check whether types are match
getSeriesSchemas(insertRowPlan);
insertRowPlan.transferType();
@@ -1351,26 +1401,66 @@
return true;
}
+ private boolean createAlignedTimeSeries(CreateAlignedTimeSeriesPlan createAlignedTimeSeriesPlan)
+ throws QueryProcessException {
+ try {
+ IoTDB.metaManager.createAlignedTimeSeries(createAlignedTimeSeriesPlan);
+ } catch (MetadataException e) {
+ throw new QueryProcessException(e);
+ }
+ return true;
+ }
+
+ @SuppressWarnings("squid:S3776") // high Cognitive Complexity
private boolean createMultiTimeSeries(CreateMultiTimeSeriesPlan multiPlan)
throws BatchProcessException {
+ int dataTypeIdx = 0;
for (int i = 0; i < multiPlan.getPaths().size(); i++) {
if (multiPlan.getResults().containsKey(i)) {
continue;
}
- CreateTimeSeriesPlan plan =
- new CreateTimeSeriesPlan(
- multiPlan.getPaths().get(i),
- multiPlan.getDataTypes().get(i),
- multiPlan.getEncodings().get(i),
- multiPlan.getCompressors().get(i),
- multiPlan.getProps() == null ? null : multiPlan.getProps().get(i),
- multiPlan.getTags() == null ? null : multiPlan.getTags().get(i),
- multiPlan.getAttributes() == null ? null : multiPlan.getAttributes().get(i),
- multiPlan.getAlias() == null ? null : multiPlan.getAlias().get(i));
- try {
- createTimeSeries(plan);
- } catch (QueryProcessException e) {
- multiPlan.getResults().put(i, RpcUtils.getStatus(e.getErrorCode(), e.getMessage()));
+ PartialPath path = multiPlan.getPaths().get(i);
+ String measurement = path.getMeasurement();
+ if (measurement.contains("(") && measurement.contains(",")) {
+ PartialPath devicePath = path.getDevicePath();
+ List<String> measurements = MetaUtils.getMeasurementsInPartialPath(path);
+ List<TSDataType> dataTypes = new ArrayList<>();
+ List<TSEncoding> encodings = new ArrayList<>();
+ for (int j = 0; j < measurements.size(); j++) {
+ dataTypes.add(multiPlan.getDataTypes().get(dataTypeIdx));
+ encodings.add(multiPlan.getEncodings().get(dataTypeIdx));
+ dataTypeIdx++;
+ }
+ CreateAlignedTimeSeriesPlan plan =
+ new CreateAlignedTimeSeriesPlan(
+ devicePath,
+ measurements,
+ dataTypes,
+ encodings,
+ multiPlan.getCompressors().get(i),
+ null);
+ try {
+ createAlignedTimeSeries(plan);
+ } catch (QueryProcessException e) {
+ multiPlan.getResults().put(i, RpcUtils.getStatus(e.getErrorCode(), e.getMessage()));
+ }
+ } else {
+ CreateTimeSeriesPlan plan =
+ new CreateTimeSeriesPlan(
+ multiPlan.getPaths().get(i),
+ multiPlan.getDataTypes().get(i),
+ multiPlan.getEncodings().get(i),
+ multiPlan.getCompressors().get(i),
+ multiPlan.getProps() == null ? null : multiPlan.getProps().get(i),
+ multiPlan.getTags() == null ? null : multiPlan.getTags().get(i),
+ multiPlan.getAttributes() == null ? null : multiPlan.getAttributes().get(i),
+ multiPlan.getAlias() == null ? null : multiPlan.getAlias().get(i));
+ dataTypeIdx++;
+ try {
+ createTimeSeries(plan);
+ } catch (QueryProcessException e) {
+ multiPlan.getResults().put(i, RpcUtils.getStatus(e.getErrorCode(), e.getMessage()));
+ }
}
}
if (!multiPlan.getResults().isEmpty()) {
@@ -1753,7 +1843,7 @@
/**
* @param storageGroups the storage groups to check
- * @return List<PartialPath> the storage groups that not exist
+ * @return List of PartialPath the storage groups that not exist
*/
List<PartialPath> checkStorageGroupExist(List<PartialPath> storageGroups) {
List<PartialPath> noExistSg = new ArrayList<>();
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
index cfa981e..e1fb10f 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
@@ -135,7 +135,9 @@
UDTF,
CREATE_FUNCTION,
DROP_FUNCTION,
+ CREATE_ALIGNED_TIMESERIES,
CREATE_MULTI_TIMESERIES,
+ AUTO_CREATE_DEVICE_MNODE,
CREATE_INDEX,
DROP_INDEX,
QUERY_INDEX,
@@ -151,6 +153,9 @@
CREATE_TRIGGER,
DROP_TRIGGER,
START_TRIGGER,
- STOP_TRIGGER
+ STOP_TRIGGER,
+ CREATE_TEMPLATE,
+ SET_DEVICE_TEMPLATE,
+ SET_USING_DEVICE_TEMPLATE,
}
}
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 784ec79..4c5b1a1 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
@@ -23,15 +23,19 @@
import org.apache.iotdb.db.metadata.PartialPath;
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.CreateTemplatePlan;
import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
import org.apache.iotdb.db.qp.physical.crud.InsertMultiTabletPlan;
import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
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.SetDeviceTemplatePlan;
import org.apache.iotdb.db.qp.physical.sys.AlterTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
+import org.apache.iotdb.db.qp.physical.sys.AutoCreateDeviceMNodePlan;
import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateIndexPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateMultiTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
@@ -47,6 +51,7 @@
import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetUsingDeviceTemplatePlan;
import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.StartTriggerPlan;
@@ -246,6 +251,9 @@
case CREATE_TIMESERIES:
plan = new CreateTimeSeriesPlan();
break;
+ case CREATE_ALIGNED_TIMESERIES:
+ plan = new CreateAlignedTimeSeriesPlan();
+ break;
case DELETE_TIMESERIES:
plan = new DeleteTimeSeriesPlan();
break;
@@ -348,6 +356,18 @@
case STOP_TRIGGER:
plan = new StopTriggerPlan();
break;
+ case CREATE_TEMPLATE:
+ plan = new CreateTemplatePlan();
+ break;
+ case SET_DEVICE_TEMPLATE:
+ plan = new SetDeviceTemplatePlan();
+ break;
+ case SET_USING_DEVICE_TEMPLATE:
+ plan = new SetUsingDeviceTemplatePlan();
+ break;
+ case AUTO_CREATE_DEVICE_MNODE:
+ plan = new AutoCreateDeviceMNodePlan();
+ break;
default:
throw new IOException("unrecognized log type " + type);
}
@@ -362,6 +382,7 @@
BATCHINSERT,
SET_STORAGE_GROUP,
CREATE_TIMESERIES,
+ CREATE_ALIGNED_TIMESERIES,
TTL,
GRANT_WATERMARK_EMBEDDING,
REVOKE_WATERMARK_EMBEDDING,
@@ -394,6 +415,10 @@
MULTI_BATCH_INSERT,
BATCH_INSERT_ROWS,
SHOW_DEVICES,
+ CREATE_TEMPLATE,
+ SET_DEVICE_TEMPLATE,
+ SET_USING_DEVICE_TEMPLATE,
+ AUTO_CREATE_DEVICE_MNODE,
CREATE_TRIGGER,
DROP_TRIGGER,
START_TRIGGER,
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/AggregationPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/AggregationPlan.java
index a9bd3e1..8a26610 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/AggregationPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/AggregationPlan.java
@@ -134,4 +134,9 @@
}
return columnForDisplay;
}
+
+ @Override
+ public boolean isRawQuery() {
+ return false;
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/CreateTemplatePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/CreateTemplatePlan.java
new file mode 100644
index 0000000..6dba97a
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/CreateTemplatePlan.java
@@ -0,0 +1,240 @@
+/*
+ * 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.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+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.utils.ReadWriteIOUtils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+public class CreateTemplatePlan extends PhysicalPlan {
+
+ String name;
+ List<List<String>> measurements;
+ List<List<TSDataType>> dataTypes;
+ List<List<TSEncoding>> encodings;
+ List<CompressionType> compressors;
+
+ public String getName() {
+ return name;
+ }
+
+ public void setName(String name) {
+ this.name = name;
+ }
+
+ public List<List<String>> getMeasurements() {
+ return measurements;
+ }
+
+ public void setMeasurements(List<List<String>> measurements) {
+ this.measurements = measurements;
+ }
+
+ public List<List<TSDataType>> getDataTypes() {
+ return dataTypes;
+ }
+
+ public void setDataTypes(List<List<TSDataType>> dataTypes) {
+ this.dataTypes = dataTypes;
+ }
+
+ public List<List<TSEncoding>> getEncodings() {
+ return encodings;
+ }
+
+ public void setEncodings(List<List<TSEncoding>> encodings) {
+ this.encodings = encodings;
+ }
+
+ public List<CompressionType> getCompressors() {
+ return compressors;
+ }
+
+ public void setCompressors(List<CompressionType> compressors) {
+ this.compressors = compressors;
+ }
+
+ public CreateTemplatePlan() {
+ super(false, OperatorType.CREATE_TEMPLATE);
+ }
+
+ public CreateTemplatePlan(
+ String name,
+ List<List<String>> measurements,
+ List<List<TSDataType>> dataTypes,
+ List<List<TSEncoding>> encodings,
+ List<CompressionType> compressors) {
+ super(false, OperatorType.CREATE_TEMPLATE);
+ this.name = name;
+ this.measurements = measurements;
+ this.dataTypes = dataTypes;
+ this.encodings = encodings;
+ this.compressors = compressors;
+ }
+
+ @Override
+ public void serialize(ByteBuffer buffer) {
+ buffer.put((byte) PhysicalPlanType.CREATE_TEMPLATE.ordinal());
+
+ ReadWriteIOUtils.write(name, buffer);
+
+ // measurements
+ ReadWriteIOUtils.write(measurements.size(), buffer);
+ for (List<String> measurementList : measurements) {
+ ReadWriteIOUtils.write(measurementList.size(), buffer);
+ for (String measurement : measurementList) {
+ ReadWriteIOUtils.write(measurement, buffer);
+ }
+ }
+
+ // datatype
+ ReadWriteIOUtils.write(dataTypes.size(), buffer);
+ for (List<TSDataType> dataTypesList : dataTypes) {
+ ReadWriteIOUtils.write(dataTypesList.size(), buffer);
+ for (TSDataType dataType : dataTypesList) {
+ ReadWriteIOUtils.write(dataType.ordinal(), buffer);
+ }
+ }
+
+ // encoding
+ ReadWriteIOUtils.write(encodings.size(), buffer);
+ for (List<TSEncoding> encodingList : encodings) {
+ ReadWriteIOUtils.write(encodingList.size(), buffer);
+ for (TSEncoding encoding : encodingList) {
+ ReadWriteIOUtils.write(encoding.ordinal(), buffer);
+ }
+ }
+
+ // compressor
+ ReadWriteIOUtils.write(compressors.size(), buffer);
+ for (CompressionType compressionType : compressors) {
+ ReadWriteIOUtils.write(compressionType.ordinal(), buffer);
+ }
+
+ buffer.putLong(index);
+ }
+
+ @Override
+ public void deserialize(ByteBuffer buffer) {
+ name = ReadWriteIOUtils.readString(buffer);
+
+ // measurements
+ int size = ReadWriteIOUtils.readInt(buffer);
+ measurements = new ArrayList<>(size);
+ for (int i = 0; i < size; i++) {
+ int listSize = ReadWriteIOUtils.readInt(buffer);
+ List<String> measurementsList = new ArrayList<>(listSize);
+ for (int j = 0; j < listSize; j++) {
+ measurementsList.add(ReadWriteIOUtils.readString(buffer));
+ }
+ measurements.add(measurementsList);
+ }
+
+ // datatypes
+ size = ReadWriteIOUtils.readInt(buffer);
+ dataTypes = new ArrayList<>(size);
+ for (int i = 0; i < size; i++) {
+ int listSize = ReadWriteIOUtils.readInt(buffer);
+ List<TSDataType> dataTypesList = new ArrayList<>(listSize);
+ for (int j = 0; j < listSize; j++) {
+ dataTypesList.add(TSDataType.values()[ReadWriteIOUtils.readInt(buffer)]);
+ }
+ dataTypes.add(dataTypesList);
+ }
+
+ // encodings
+ size = ReadWriteIOUtils.readInt(buffer);
+ encodings = new ArrayList<>(size);
+ for (int i = 0; i < size; i++) {
+ int listSize = ReadWriteIOUtils.readInt(buffer);
+ List<TSEncoding> encodingsList = new ArrayList<>(listSize);
+ for (int j = 0; j < listSize; j++) {
+ encodingsList.add(TSEncoding.values()[ReadWriteIOUtils.readInt(buffer)]);
+ }
+ encodings.add(encodingsList);
+ }
+
+ // compressor
+ size = ReadWriteIOUtils.readInt(buffer);
+ compressors = new ArrayList<>(size);
+ for (int i = 0; i < size; i++) {
+ compressors.add(CompressionType.values()[ReadWriteIOUtils.readInt(buffer)]);
+ }
+
+ this.index = buffer.getLong();
+ }
+
+ @Override
+ public void serialize(DataOutputStream stream) throws IOException {
+ stream.writeByte((byte) PhysicalPlanType.CREATE_TEMPLATE.ordinal());
+
+ ReadWriteIOUtils.write(name, stream);
+
+ // measurements
+ ReadWriteIOUtils.write(measurements.size(), stream);
+ for (List<String> measurementList : measurements) {
+ ReadWriteIOUtils.write(measurementList.size(), stream);
+ for (String measurement : measurementList) {
+ ReadWriteIOUtils.write(measurement, stream);
+ }
+ }
+
+ // datatype
+ ReadWriteIOUtils.write(dataTypes.size(), stream);
+ for (List<TSDataType> dataTypesList : dataTypes) {
+ ReadWriteIOUtils.write(dataTypesList.size(), stream);
+ for (TSDataType dataType : dataTypesList) {
+ ReadWriteIOUtils.write(dataType.ordinal(), stream);
+ }
+ }
+
+ // encoding
+ ReadWriteIOUtils.write(encodings.size(), stream);
+ for (List<TSEncoding> encodingList : encodings) {
+ ReadWriteIOUtils.write(encodingList.size(), stream);
+ for (TSEncoding encoding : encodingList) {
+ ReadWriteIOUtils.write(encoding.ordinal(), stream);
+ }
+ }
+
+ // compressor
+ ReadWriteIOUtils.write(compressors.size(), stream);
+ for (CompressionType compressionType : compressors) {
+ ReadWriteIOUtils.write(compressionType.ordinal(), stream);
+ }
+
+ stream.writeLong(index);
+ }
+
+ @Override
+ public List<PartialPath> getPaths() {
+ return null;
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/FillQueryPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/FillQueryPlan.java
index c072c9f..c40274a 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/FillQueryPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/FillQueryPlan.java
@@ -58,4 +58,9 @@
getOperatorType().name() + " doesn't support disable align clause.");
}
}
+
+ @Override
+ public boolean isRawQuery() {
+ return false;
+ }
}
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
index 8399527..bd0150d 100644
--- 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
@@ -53,6 +53,7 @@
private static final Logger logger = LoggerFactory.getLogger(InsertRowPlan.class);
private static final byte TYPE_RAW_STRING = -1;
+ private static final byte TYPE_NULL = -2;
private long time;
private Object[] values;
@@ -85,11 +86,11 @@
this.time = insertTime;
this.deviceId = deviceId;
this.measurements = measurementList;
- this.dataTypes = new TSDataType[measurements.length];
+ 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[measurements.length];
- System.arraycopy(insertValues, 0, values, 0, measurements.length);
+ this.values = new Object[insertValues.length];
+ System.arraycopy(insertValues, 0, values, 0, insertValues.length);
isNeedInferType = true;
}
@@ -118,8 +119,8 @@
this.deviceId = deviceId;
this.measurements = measurements;
this.dataTypes = dataTypes;
- this.values = new Object[measurements.length];
- for (int i = 0; i < measurements.length; i++) {
+ 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) {
@@ -194,6 +195,7 @@
@SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
public void transferType() throws QueryProcessException {
if (isNeedInferType) {
+ int columnIndex = 0;
for (int i = 0; i < measurementMNodes.length; i++) {
if (measurementMNodes[i] == null) {
if (IoTDBDescriptor.getInstance().getConfig().isEnablePartialInsert()) {
@@ -207,23 +209,52 @@
new PathNotExistException(
deviceId.getFullPath() + IoTDBConstant.PATH_SEPARATOR + measurements[i]));
}
+ columnIndex++;
continue;
}
- dataTypes[i] = measurementMNodes[i].getSchema().getType();
- try {
- values[i] = CommonUtils.parseValue(dataTypes[i], values[i].toString());
- } catch (Exception e) {
- logger.warn(
- "{}.{} data type is not consistent, input {}, registered {}",
- deviceId,
- measurements[i],
- values[i],
- dataTypes[i]);
- if (IoTDBDescriptor.getInstance().getConfig().isEnablePartialInsert()) {
- markFailedMeasurementInsertion(i, e);
- measurementMNodes[i] = null;
- } else {
- throw e;
+ if (measurementMNodes[i].getSchema().getType() != TSDataType.VECTOR) {
+ dataTypes[columnIndex] = measurementMNodes[i].getSchema().getType();
+ try {
+ values[columnIndex] =
+ CommonUtils.parseValue(dataTypes[columnIndex], values[columnIndex].toString());
+ } catch (Exception e) {
+ logger.warn(
+ "{}.{} data type is not consistent, input {}, registered {}",
+ deviceId,
+ measurements[i],
+ values[i],
+ dataTypes[i]);
+ if (IoTDBDescriptor.getInstance().getConfig().isEnablePartialInsert()) {
+ markFailedMeasurementInsertion(i, e);
+ measurementMNodes[i] = null;
+ } else {
+ throw e;
+ }
+ }
+ columnIndex++;
+ }
+ // for aligned timeseries
+ else {
+ for (TSDataType dataType : measurementMNodes[i].getSchema().getValueTSDataTypeList()) {
+ dataTypes[columnIndex] = dataType;
+ try {
+ values[columnIndex] =
+ CommonUtils.parseValue(dataTypes[columnIndex], values[columnIndex].toString());
+ } catch (Exception e) {
+ logger.warn(
+ "{}.{} data type is not consistent, input {}, registered {}",
+ deviceId,
+ measurements[i],
+ values[columnIndex],
+ dataTypes[columnIndex]);
+ if (IoTDBDescriptor.getInstance().getConfig().isEnablePartialInsert()) {
+ markFailedMeasurementInsertion(i, e);
+ measurementMNodes[i] = null;
+ } else {
+ throw e;
+ }
+ }
+ columnIndex++;
}
}
}
@@ -246,6 +277,7 @@
}
failedValues.add(values[index]);
values[index] = null;
+ dataTypes[index] = null;
}
@Override
@@ -310,6 +342,7 @@
}
try {
+ stream.writeInt(dataTypes.length);
putValues(stream);
} catch (QueryProcessException e) {
throw new IOException(e);
@@ -323,7 +356,8 @@
private void putValues(DataOutputStream outputStream) throws QueryProcessException, IOException {
for (int i = 0; i < values.length; i++) {
- if (measurements[i] == null) {
+ if (values[i] == null) {
+ ReadWriteIOUtils.write(TYPE_NULL, outputStream);
continue;
}
// types are not determined, the situation mainly occurs when the plan uses string values
@@ -361,7 +395,8 @@
private void putValues(ByteBuffer buffer) throws QueryProcessException {
for (int i = 0; i < values.length; i++) {
- if (measurements[i] == null) {
+ if (values[i] == null) {
+ ReadWriteIOUtils.write(TYPE_NULL, buffer);
continue;
}
// types are not determined, the situation mainly occurs when the plan uses string values
@@ -399,15 +434,14 @@
/** Make sure the values is already inited before calling this */
public void fillValues(ByteBuffer buffer) throws QueryProcessException {
- for (int i = 0; i < measurements.length; i++) {
+ 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);
+ if (typeNum == TYPE_RAW_STRING || typeNum == TYPE_NULL) {
+ values[i] = typeNum == TYPE_RAW_STRING ? ReadWriteIOUtils.readString(buffer) : null;
continue;
}
-
dataTypes[i] = TSDataType.values()[typeNum];
switch (dataTypes[i]) {
case BOOLEAN:
@@ -456,8 +490,8 @@
putString(buffer, measurement);
}
}
-
try {
+ buffer.putInt(dataTypes.length);
putValues(buffer);
} catch (QueryProcessException e) {
logger.error("Failed to serialize values for {}", this, e);
@@ -483,8 +517,9 @@
measurements[i] = readString(buffer);
}
- this.dataTypes = new TSDataType[measurementSize];
- this.values = new Object[measurementSize];
+ int dataTypeSize = buffer.getInt();
+ this.dataTypes = new TSDataType[dataTypeSize];
+ this.values = new Object[dataTypeSize];
try {
fillValues(buffer);
} catch (QueryProcessException e) {
@@ -556,12 +591,6 @@
if (values.length == 0) {
throw new QueryProcessException("The size of values is 0");
}
- if (measurements.length != values.length) {
- throw new QueryProcessException(
- String.format(
- "Measurements length [%d] does not match " + "values length [%d]",
- measurements.length, values.length));
- }
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/InsertTabletPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertTabletPlan.java
index fa75882..267c28c 100644
--- 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
@@ -27,6 +27,7 @@
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.TsPrimitiveType;
import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsBinary;
@@ -52,6 +53,7 @@
private long[] times; // times should be sorted. It is done in the session API.
private ByteBuffer timeBuffer;
+ private BitMap[] bitMaps;
private Object[] columns;
private ByteBuffer valueBuffer;
private int rowCount = 0;
@@ -152,6 +154,7 @@
writeMeasurements(stream);
writeDataTypes(stream);
writeTimes(stream);
+ writeBitMaps(stream);
writeValues(stream);
}
@@ -167,8 +170,9 @@
}
private void writeDataTypes(DataOutputStream stream) throws IOException {
+ stream.writeInt(dataTypes.length);
for (int i = 0; i < dataTypes.length; i++) {
- if (measurements[i] == null) {
+ if (columns[i] == null) {
continue;
}
TSDataType dataType = dataTypes[i];
@@ -199,6 +203,20 @@
}
}
+ private void writeBitMaps(DataOutputStream stream) throws IOException {
+ stream.writeBoolean(bitMaps != null);
+ if (bitMaps != null) {
+ for (BitMap bitMap : bitMaps) {
+ if (bitMap == null) {
+ stream.writeBoolean(false);
+ } else {
+ stream.writeBoolean(true);
+ stream.write(bitMap.getByteArray());
+ }
+ }
+ }
+ }
+
private void writeValues(DataOutputStream stream) throws IOException {
if (valueBuffer == null) {
serializeValues(stream);
@@ -222,6 +240,7 @@
writeMeasurements(buffer);
writeDataTypes(buffer);
writeTimes(buffer);
+ writeBitMaps(buffer);
writeValues(buffer);
}
@@ -236,11 +255,13 @@
}
private void writeDataTypes(ByteBuffer buffer) {
+ buffer.putInt(dataTypes.length - (failedMeasurements == null ? 0 : failedMeasurements.size()));
for (int i = 0, dataTypesLength = dataTypes.length; i < dataTypesLength; i++) {
TSDataType dataType = dataTypes[i];
- if (measurements[i] != null) {
- dataType.serializeTo(buffer);
+ if (columns[i] == null) {
+ continue;
}
+ dataType.serializeTo(buffer);
}
}
@@ -267,6 +288,20 @@
}
}
+ private void writeBitMaps(ByteBuffer buffer) {
+ buffer.put(BytesUtils.boolToByte(bitMaps != null));
+ if (bitMaps != null) {
+ for (BitMap bitMap : bitMaps) {
+ if (bitMap == null) {
+ buffer.put(BytesUtils.boolToByte(false));
+ } else {
+ buffer.put(BytesUtils.boolToByte(true));
+ buffer.put(bitMap.getByteArray());
+ }
+ }
+ }
+ }
+
private void writeValues(ByteBuffer buffer) {
if (valueBuffer == null) {
serializeValues(buffer);
@@ -279,8 +314,8 @@
}
private void serializeValues(DataOutputStream outputStream) throws IOException {
- for (int i = 0; i < measurements.length; i++) {
- if (measurements[i] == null) {
+ for (int i = 0; i < dataTypes.length; i++) {
+ if (columns[i] == null) {
continue;
}
serializeColumn(dataTypes[i], columns[i], outputStream, start, end);
@@ -288,8 +323,8 @@
}
private void serializeValues(ByteBuffer buffer) {
- for (int i = 0; i < measurements.length; i++) {
- if (measurements[i] == null) {
+ for (int i = 0; i < dataTypes.length; i++) {
+ if (columns[i] == null) {
continue;
}
serializeColumn(dataTypes[i], columns[i], buffer, start, end);
@@ -411,8 +446,9 @@
measurements[i] = readString(buffer);
}
- this.dataTypes = new TSDataType[measurementSize];
- for (int i = 0; i < measurementSize; i++) {
+ int dataTypeSize = buffer.getInt();
+ this.dataTypes = new TSDataType[dataTypeSize];
+ for (int i = 0; i < dataTypeSize; i++) {
dataTypes[i] = TSDataType.deserialize(buffer.get());
}
@@ -421,7 +457,11 @@
this.times = new long[rows];
times = QueryDataSetUtils.readTimesFromBuffer(buffer, rows);
- columns = QueryDataSetUtils.readValuesFromBuffer(buffer, dataTypes, measurementSize, rows);
+ boolean hasBitMaps = BytesUtils.byteToBool(buffer.get());
+ if (hasBitMaps) {
+ bitMaps = QueryDataSetUtils.readBitMapsFromBuffer(buffer, dataTypeSize, rows);
+ }
+ columns = QueryDataSetUtils.readValuesFromBuffer(buffer, dataTypes, dataTypeSize, rows);
this.index = buffer.getLong();
}
@@ -444,6 +484,14 @@
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;
@@ -589,7 +637,7 @@
if (columns == null || columns.length == 0) {
throw new QueryProcessException("Values are null");
}
- if (measurements.length != columns.length) {
+ if (dataTypes.length != columns.length) {
throw new QueryProcessException(
String.format(
"Measurements length [%d] does not match " + "columns length [%d]",
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/LastQueryPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/LastQueryPlan.java
index b657a49..56fdd1c 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/LastQueryPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/LastQueryPlan.java
@@ -53,4 +53,9 @@
}
return false;
}
+
+ @Override
+ public boolean isRawQuery() {
+ return false;
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/QueryIndexPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/QueryIndexPlan.java
index 27d20f1..8a931bf 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/QueryIndexPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/QueryIndexPlan.java
@@ -73,4 +73,9 @@
public String toString() {
return String.format("Query paths: %s, index type: %s, props: %s", paths, indexType, props);
}
+
+ @Override
+ public boolean isRawQuery() {
+ return false;
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/QueryPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/QueryPlan.java
index a188db5..2de90a5 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/QueryPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/QueryPlan.java
@@ -42,6 +42,8 @@
private Map<String, Integer> pathToIndex = new HashMap<>();
+ private Map<String, Integer> vectorPathToIndex = new HashMap<>();
+
private boolean enableRedirect = false;
public QueryPlan() {
@@ -103,6 +105,10 @@
pathToIndex.put(columnName, index);
}
+ public void setPathToIndex(Map<String, Integer> pathToIndex) {
+ this.pathToIndex = pathToIndex;
+ }
+
public Map<String, Integer> getPathToIndex() {
return pathToIndex;
}
@@ -136,4 +142,12 @@
public void setEnableRedirect(boolean enableRedirect) {
this.enableRedirect = enableRedirect;
}
+
+ public Map<String, Integer> getVectorPathToIndex() {
+ return vectorPathToIndex;
+ }
+
+ public void setVectorPathToIndex(Map<String, Integer> vectorPathToIndex) {
+ this.vectorPathToIndex = vectorPathToIndex;
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/RawDataQueryPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/RawDataQueryPlan.java
index fc975df..6dc4d60 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/RawDataQueryPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/RawDataQueryPlan.java
@@ -18,8 +18,11 @@
*/
package org.apache.iotdb.db.qp.physical.crud;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.MManager;
import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.VectorPartialPath;
import org.apache.iotdb.db.qp.logical.Operator;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.common.Path;
@@ -40,6 +43,9 @@
private IExpression expression = null;
private Map<String, Set<String>> deviceToMeasurements = new HashMap<>();
+ private List<PartialPath> deduplicatedVectorPaths = new ArrayList<>();
+ private List<TSDataType> deduplicatedVectorDataTypes = new ArrayList<>();
+
public RawDataQueryPlan() {
super();
}
@@ -71,13 +77,23 @@
* used for AlignByDevice Query, the query is executed by each device, So we only maintain
* measurements of current device.
*/
- public void setDeduplicatedPaths(List<PartialPath> deduplicatedPaths) {
+ public void setDeduplicatedPathsAndUpdate(List<PartialPath> deduplicatedPaths) {
deviceToMeasurements.clear();
deduplicatedPaths.forEach(
- path ->
- deviceToMeasurements
- .computeIfAbsent(path.getDevice(), key -> new HashSet<>())
- .add(path.getMeasurement()));
+ path -> {
+ Set<String> set =
+ deviceToMeasurements.computeIfAbsent(path.getDevice(), key -> new HashSet<>());
+ set.add(path.getMeasurement());
+ if (path instanceof VectorPartialPath) {
+ ((VectorPartialPath) path)
+ .getSubSensorsPathList()
+ .forEach(subSensor -> set.add(subSensor.getMeasurement()));
+ }
+ });
+ this.deduplicatedPaths = deduplicatedPaths;
+ }
+
+ public void setDeduplicatedPaths(List<PartialPath> deduplicatedPaths) {
this.deduplicatedPaths = deduplicatedPaths;
}
@@ -106,4 +122,41 @@
public Map<String, Set<String>> getDeviceToMeasurements() {
return deviceToMeasurements;
}
+
+ public void transformPaths(MManager mManager) throws MetadataException {
+ for (int i = 0; i < deduplicatedPaths.size(); i++) {
+ PartialPath path = mManager.transformPath(deduplicatedPaths.get(i));
+ if (path instanceof VectorPartialPath) {
+ deduplicatedPaths.set(i, path);
+ }
+ }
+ }
+
+ public List<PartialPath> getDeduplicatedVectorPaths() {
+ return deduplicatedVectorPaths;
+ }
+
+ public void setDeduplicatedVectorPaths(List<PartialPath> deduplicatedVectorPaths) {
+ this.deduplicatedVectorPaths = deduplicatedVectorPaths;
+ }
+
+ public List<TSDataType> getDeduplicatedVectorDataTypes() {
+ return deduplicatedVectorDataTypes;
+ }
+
+ public void setDeduplicatedVectorDataTypes(List<TSDataType> deduplicatedVectorDataTypes) {
+ this.deduplicatedVectorDataTypes = deduplicatedVectorDataTypes;
+ }
+
+ public void transformToVector() {
+ if (!this.deduplicatedVectorPaths.isEmpty()) {
+ this.deduplicatedPaths = this.deduplicatedVectorPaths;
+ this.deduplicatedDataTypes = this.deduplicatedVectorDataTypes;
+ setPathToIndex(getVectorPathToIndex());
+ }
+ }
+
+ public boolean isRawQuery() {
+ return true;
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/SetDeviceTemplatePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/SetDeviceTemplatePlan.java
new file mode 100644
index 0000000..b76703a
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/SetDeviceTemplatePlan.java
@@ -0,0 +1,94 @@
+/*
+ * 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.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+public class SetDeviceTemplatePlan extends PhysicalPlan {
+ String templateName;
+ String prefixPath;
+
+ public SetDeviceTemplatePlan() {
+ super(false, OperatorType.SET_DEVICE_TEMPLATE);
+ }
+
+ public SetDeviceTemplatePlan(String templateName, String prefixPath) {
+ super(false, OperatorType.SET_DEVICE_TEMPLATE);
+ this.templateName = templateName;
+ this.prefixPath = prefixPath;
+ }
+
+ public String getTemplateName() {
+ return templateName;
+ }
+
+ public void setTemplateName(String templateName) {
+ this.templateName = templateName;
+ }
+
+ public String getPrefixPath() {
+ return prefixPath;
+ }
+
+ public void setPrefixPath(String prefixPath) {
+ this.prefixPath = prefixPath;
+ }
+
+ @Override
+ public List<PartialPath> getPaths() {
+ return null;
+ }
+
+ @Override
+ public void serialize(ByteBuffer buffer) {
+ buffer.put((byte) PhysicalPlanType.SET_DEVICE_TEMPLATE.ordinal());
+
+ ReadWriteIOUtils.write(templateName, buffer);
+ ReadWriteIOUtils.write(prefixPath, buffer);
+
+ buffer.putLong(index);
+ }
+
+ @Override
+ public void deserialize(ByteBuffer buffer) {
+ templateName = ReadWriteIOUtils.readString(buffer);
+ prefixPath = ReadWriteIOUtils.readString(buffer);
+
+ this.index = buffer.getLong();
+ }
+
+ @Override
+ public void serialize(DataOutputStream stream) throws IOException {
+ stream.writeByte((byte) PhysicalPlanType.SET_DEVICE_TEMPLATE.ordinal());
+
+ ReadWriteIOUtils.write(templateName, stream);
+ ReadWriteIOUtils.write(prefixPath, stream);
+
+ stream.writeLong(index);
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/UDTFPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/UDTFPlan.java
index 5192bbc..78eb25d 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/UDTFPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/UDTFPlan.java
@@ -138,4 +138,9 @@
}
return columnForReader;
}
+
+ @Override
+ public boolean isRawQuery() {
+ return false;
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/AutoCreateDeviceMNodePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/AutoCreateDeviceMNodePlan.java
new file mode 100644
index 0000000..ef7412e
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/AutoCreateDeviceMNodePlan.java
@@ -0,0 +1,87 @@
+/*
+ * 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.sys;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+
+public class AutoCreateDeviceMNodePlan extends PhysicalPlan {
+
+ private static final Logger logger = LoggerFactory.getLogger(AutoCreateDeviceMNodePlan.class);
+ protected PartialPath path;
+
+ public AutoCreateDeviceMNodePlan() {
+ super(false, Operator.OperatorType.AUTO_CREATE_DEVICE_MNODE);
+ }
+
+ public AutoCreateDeviceMNodePlan(PartialPath path) {
+ super(false, Operator.OperatorType.AUTO_CREATE_DEVICE_MNODE);
+ this.path = path;
+ }
+
+ public AutoCreateDeviceMNodePlan(boolean isQuery, Operator.OperatorType operatorType) {
+ super(isQuery, operatorType);
+ }
+
+ @Override
+ public List<PartialPath> getPaths() {
+ return Collections.singletonList(path);
+ }
+
+ public PartialPath getPath() {
+ return path;
+ }
+
+ @Override
+ public void serialize(ByteBuffer buffer) {
+ buffer.put((byte) PhysicalPlanType.AUTO_CREATE_DEVICE_MNODE.ordinal());
+ putString(buffer, path.getFullPath());
+ buffer.putLong(index);
+ }
+
+ @Override
+ public void serialize(DataOutputStream stream) throws IOException {
+ stream.write((byte) PhysicalPlanType.AUTO_CREATE_DEVICE_MNODE.ordinal());
+ putString(stream, path.getFullPath());
+ stream.writeLong(index);
+ }
+
+ @Override
+ public void deserialize(ByteBuffer buffer) {
+ String pathString = readString(buffer);
+ try {
+ path = new PartialPath(pathString);
+ } catch (IllegalPathException e) {
+ logger.error("Failed to deserialize device {} from buffer", pathString);
+ }
+ index = buffer.getLong();
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateAlignedTimeSeriesPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateAlignedTimeSeriesPlan.java
new file mode 100644
index 0000000..5451860
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateAlignedTimeSeriesPlan.java
@@ -0,0 +1,259 @@
+/*
+ * 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.sys;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+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.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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 CreateAlignedTimeSeriesPlan extends PhysicalPlan {
+
+ private static final Logger logger = LoggerFactory.getLogger(CreateAlignedTimeSeriesPlan.class);
+
+ private PartialPath devicePath;
+ private List<String> measurements;
+ private List<TSDataType> dataTypes;
+ private List<TSEncoding> encodings;
+ private CompressionType compressor;
+ private List<String> aliasList;
+
+ public CreateAlignedTimeSeriesPlan() {
+ super(false, Operator.OperatorType.CREATE_ALIGNED_TIMESERIES);
+ canBeSplit = false;
+ }
+
+ public CreateAlignedTimeSeriesPlan(
+ PartialPath devicePath,
+ List<String> measurements,
+ List<TSDataType> dataTypes,
+ List<TSEncoding> encodings,
+ CompressionType compressor,
+ List<String> aliasList) {
+ super(false, Operator.OperatorType.CREATE_ALIGNED_TIMESERIES);
+ this.devicePath = devicePath;
+ this.measurements = measurements;
+ this.dataTypes = dataTypes;
+ this.encodings = encodings;
+ this.compressor = compressor;
+ this.aliasList = aliasList;
+ this.canBeSplit = false;
+ }
+
+ public PartialPath getDevicePath() {
+ return devicePath;
+ }
+
+ public void setDevicePath(PartialPath devicePath) {
+ this.devicePath = devicePath;
+ }
+
+ public List<String> getMeasurements() {
+ return measurements;
+ }
+
+ public void setMeasurements(List<String> measurements) {
+ this.measurements = measurements;
+ }
+
+ public List<TSDataType> getDataTypes() {
+ return dataTypes;
+ }
+
+ public void setDataTypes(List<TSDataType> dataTypes) {
+ this.dataTypes = dataTypes;
+ }
+
+ public List<TSEncoding> getEncodings() {
+ return encodings;
+ }
+
+ public void setEncodings(List<TSEncoding> encodings) {
+ this.encodings = encodings;
+ }
+
+ public CompressionType getCompressor() {
+ return compressor;
+ }
+
+ public void setCompressor(CompressionType compressor) {
+ this.compressor = compressor;
+ }
+
+ public List<String> getAliasList() {
+ return aliasList;
+ }
+
+ public void setAliasList(List<String> aliasList) {
+ this.aliasList = aliasList;
+ }
+
+ @Override
+ public String toString() {
+ return String.format(
+ "devicePath: %s, measurements: %s, dataTypes: %s, encodings: %s, compression: %s",
+ devicePath, measurements, dataTypes, encodings, compressor);
+ }
+
+ @Override
+ public List<PartialPath> getPaths() {
+ List<PartialPath> paths = new ArrayList<>();
+ for (String measurement : measurements) {
+ try {
+ paths.add(new PartialPath(devicePath.getFullPath(), measurement));
+ } catch (IllegalPathException e) {
+ logger.error("Failed to get paths of CreateAlignedTimeSeriesPlan. ", e);
+ }
+ }
+ return paths;
+ }
+
+ @Override
+ public void serialize(DataOutputStream stream) throws IOException {
+ stream.writeByte((byte) PhysicalPlanType.CREATE_ALIGNED_TIMESERIES.ordinal());
+ byte[] bytes = devicePath.getFullPath().getBytes();
+ stream.writeInt(bytes.length);
+ stream.write(bytes);
+
+ ReadWriteIOUtils.write(measurements.size(), stream);
+ for (String measurement : measurements) {
+ ReadWriteIOUtils.write(measurement, stream);
+ }
+ for (TSDataType dataType : dataTypes) {
+ stream.write(dataType.ordinal());
+ }
+ for (TSEncoding encoding : encodings) {
+ stream.write(encoding.ordinal());
+ }
+ stream.write(compressor.ordinal());
+
+ // alias
+ if (aliasList != null) {
+ stream.write(1);
+ for (String alias : aliasList) {
+ ReadWriteIOUtils.write(alias, stream);
+ }
+ } else {
+ stream.write(0);
+ }
+ stream.writeLong(index);
+ }
+
+ @Override
+ public void serialize(ByteBuffer buffer) {
+ buffer.put((byte) PhysicalPlanType.CREATE_ALIGNED_TIMESERIES.ordinal());
+ byte[] bytes = devicePath.getFullPath().getBytes();
+ buffer.putInt(bytes.length);
+ buffer.put(bytes);
+
+ ReadWriteIOUtils.write(measurements.size(), buffer);
+ for (String measurement : measurements) {
+ ReadWriteIOUtils.write(measurement, buffer);
+ }
+ for (TSDataType dataType : dataTypes) {
+ buffer.put((byte) dataType.ordinal());
+ }
+ for (TSEncoding encoding : encodings) {
+ buffer.put((byte) encoding.ordinal());
+ }
+ buffer.put((byte) compressor.ordinal());
+
+ // alias
+ if (aliasList != null) {
+ buffer.put((byte) 1);
+ for (String alias : aliasList) {
+ ReadWriteIOUtils.write(alias, buffer);
+ }
+ } else {
+ buffer.put((byte) 0);
+ }
+
+ buffer.putLong(index);
+ }
+
+ @Override
+ public void deserialize(ByteBuffer buffer) throws IllegalPathException {
+ int length = buffer.getInt();
+ byte[] bytes = new byte[length];
+ buffer.get(bytes);
+
+ devicePath = new PartialPath(new String(bytes));
+ int size = ReadWriteIOUtils.readInt(buffer);
+ measurements = new ArrayList<>();
+ for (int i = 0; i < size; i++) {
+ measurements.add(ReadWriteIOUtils.readString(buffer));
+ }
+ dataTypes = new ArrayList<>();
+ for (int i = 0; i < size; i++) {
+ dataTypes.add(TSDataType.values()[buffer.get()]);
+ }
+ encodings = new ArrayList<>();
+ for (int i = 0; i < size; i++) {
+ encodings.add(TSEncoding.values()[buffer.get()]);
+ }
+ compressor = CompressionType.values()[buffer.get()];
+
+ // alias
+ if (buffer.get() == 1) {
+ aliasList = new ArrayList<>();
+ for (int i = 0; i < size; i++) {
+ aliasList.add(ReadWriteIOUtils.readString(buffer));
+ }
+ }
+
+ this.index = buffer.getLong();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ CreateAlignedTimeSeriesPlan that = (CreateAlignedTimeSeriesPlan) o;
+
+ return Objects.equals(devicePath, that.devicePath)
+ && Objects.equals(measurements, that.measurements)
+ && Objects.equals(dataTypes, that.dataTypes)
+ && Objects.equals(encodings, that.encodings)
+ && compressor == that.compressor;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(devicePath, measurements, dataTypes, encodings, compressor);
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateMultiTimeSeriesPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateMultiTimeSeriesPlan.java
index 4c0e854..e1982e6 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateMultiTimeSeriesPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateMultiTimeSeriesPlan.java
@@ -154,6 +154,7 @@
int type = PhysicalPlanType.CREATE_MULTI_TIMESERIES.ordinal();
stream.write(type);
stream.writeInt(paths.size());
+ stream.writeInt(dataTypes.size()); // size of datatypes, encodings for aligned timeseries
for (PartialPath path : paths) {
putString(stream, path.getFullPath());
@@ -211,6 +212,7 @@
int type = PhysicalPlanType.CREATE_MULTI_TIMESERIES.ordinal();
buffer.put((byte) type);
buffer.putInt(paths.size());
+ buffer.putInt(dataTypes.size()); // size of datatypes, encodings for aligned timeseries
for (PartialPath path : paths) {
putString(buffer, path.getFullPath());
@@ -266,16 +268,17 @@
@Override
public void deserialize(ByteBuffer buffer) throws IllegalPathException {
int totalSize = buffer.getInt();
+ int dataTypeSize = buffer.getInt();
paths = new ArrayList<>(totalSize);
for (int i = 0; i < totalSize; i++) {
paths.add(new PartialPath(readString(buffer)));
}
dataTypes = new ArrayList<>(totalSize);
- for (int i = 0; i < totalSize; i++) {
+ for (int i = 0; i < dataTypeSize; i++) {
dataTypes.add(TSDataType.values()[buffer.get()]);
}
encodings = new ArrayList<>(totalSize);
- for (int i = 0; i < totalSize; i++) {
+ for (int i = 0; i < dataTypeSize; i++) {
encodings.add(TSEncoding.values()[buffer.get()]);
}
compressors = new ArrayList<>(totalSize);
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/MeasurementMNodePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/MeasurementMNodePlan.java
index cee286a..a46f8f7 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/MeasurementMNodePlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/MeasurementMNodePlan.java
@@ -21,6 +21,7 @@
import org.apache.iotdb.db.metadata.PartialPath;
import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import java.io.DataOutputStream;
@@ -31,7 +32,7 @@
import java.util.Objects;
public class MeasurementMNodePlan extends MNodePlan {
- private MeasurementSchema schema;
+ private IMeasurementSchema schema;
private String alias;
private long offset;
@@ -40,7 +41,7 @@
}
public MeasurementMNodePlan(
- String name, String alias, long offset, int childSize, MeasurementSchema schema) {
+ String name, String alias, long offset, int childSize, IMeasurementSchema schema) {
super(false, Operator.OperatorType.MEASUREMENT_MNODE);
this.name = name;
this.alias = alias;
@@ -91,11 +92,11 @@
index = buffer.getLong();
}
- public MeasurementSchema getSchema() {
+ public IMeasurementSchema getSchema() {
return schema;
}
- public void setSchema(MeasurementSchema schema) {
+ public void setSchema(IMeasurementSchema schema) {
this.schema = schema;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetUsingDeviceTemplatePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetUsingDeviceTemplatePlan.java
new file mode 100644
index 0000000..6d20145
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetUsingDeviceTemplatePlan.java
@@ -0,0 +1,83 @@
+/*
+ * 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.sys;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+public class SetUsingDeviceTemplatePlan extends PhysicalPlan {
+
+ private static final Logger logger = LoggerFactory.getLogger(SetUsingDeviceTemplatePlan.class);
+ PartialPath prefixPath;
+
+ public SetUsingDeviceTemplatePlan() {
+ super(false, OperatorType.SET_USING_DEVICE_TEMPLATE);
+ }
+
+ public SetUsingDeviceTemplatePlan(PartialPath prefixPath) {
+ super(false, OperatorType.SET_USING_DEVICE_TEMPLATE);
+ this.prefixPath = prefixPath;
+ }
+
+ @Override
+ public List<PartialPath> getPaths() {
+ return null;
+ }
+
+ public PartialPath getPrefixPath() {
+ return prefixPath;
+ }
+
+ @Override
+ public void serialize(ByteBuffer buffer) {
+ buffer.put((byte) PhysicalPlanType.SET_USING_DEVICE_TEMPLATE.ordinal());
+ ReadWriteIOUtils.write(prefixPath.getFullPath(), buffer);
+ buffer.putLong(index);
+ }
+
+ @Override
+ public void deserialize(ByteBuffer buffer) {
+ String pathString = readString(buffer);
+ try {
+ prefixPath = new PartialPath(pathString);
+ } catch (IllegalPathException e) {
+ logger.error("Failed to deserialize device {} from buffer", pathString);
+ }
+ index = buffer.getLong();
+ }
+
+ @Override
+ public void serialize(DataOutputStream stream) throws IOException {
+ stream.writeByte((byte) PhysicalPlanType.SET_USING_DEVICE_TEMPLATE.ordinal());
+ ReadWriteIOUtils.write(prefixPath.getFullPath(), stream);
+ stream.writeLong(index);
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/StorageGroupMNodePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/StorageGroupMNodePlan.java
index 64f0153..89c00b7 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/StorageGroupMNodePlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/StorageGroupMNodePlan.java
@@ -32,15 +32,19 @@
public class StorageGroupMNodePlan extends MNodePlan {
private long dataTTL;
+ private int alignedTimeseriesIndex;
+
public StorageGroupMNodePlan() {
super(false, Operator.OperatorType.STORAGE_GROUP_MNODE);
}
- public StorageGroupMNodePlan(String name, long dataTTL, int childSize) {
+ public StorageGroupMNodePlan(
+ String name, long dataTTL, int childSize, int alignedTimeseriesIndex) {
super(false, Operator.OperatorType.STORAGE_GROUP_MNODE);
this.name = name;
this.dataTTL = dataTTL;
this.childSize = childSize;
+ this.alignedTimeseriesIndex = alignedTimeseriesIndex;
}
@Override
@@ -56,12 +60,21 @@
this.dataTTL = dataTTL;
}
+ public int getAlignedTimeseriesIndex() {
+ return alignedTimeseriesIndex;
+ }
+
+ public void setAlignedTimeseriesIndex(int alignedTimeseriesIndex) {
+ this.alignedTimeseriesIndex = alignedTimeseriesIndex;
+ }
+
@Override
public void serialize(ByteBuffer buffer) {
buffer.put((byte) PhysicalPlanType.STORAGE_GROUP_MNODE.ordinal());
putString(buffer, name);
buffer.putLong(dataTTL);
buffer.putInt(childSize);
+ buffer.putInt(alignedTimeseriesIndex);
buffer.putLong(index);
}
@@ -72,6 +85,7 @@
putString(stream, name);
stream.writeLong(dataTTL);
stream.writeInt(childSize);
+ stream.writeInt(alignedTimeseriesIndex);
stream.writeLong(index);
}
@@ -81,12 +95,25 @@
name = readString(buffer);
dataTTL = buffer.getLong();
childSize = buffer.getInt();
+ if (buffer.hasRemaining()) {
+ alignedTimeseriesIndex = buffer.getInt();
+ } else {
+ alignedTimeseriesIndex = 0;
+ }
index = buffer.getLong();
}
@Override
public String toString() {
- return "StorageGroupMNode{" + name + "," + dataTTL + "," + childSize + "}";
+ return "StorageGroupMNode{"
+ + name
+ + ","
+ + dataTTL
+ + ","
+ + childSize
+ + ","
+ + alignedTimeseriesIndex
+ + "}";
}
@Override
@@ -100,11 +127,12 @@
StorageGroupMNodePlan that = (StorageGroupMNodePlan) o;
return Objects.equals(name, that.name)
&& Objects.equals(dataTTL, that.dataTTL)
- && Objects.equals(childSize, that.childSize);
+ && Objects.equals(childSize, that.childSize)
+ && Objects.equals(alignedTimeseriesIndex, that.alignedTimeseriesIndex);
}
@Override
public int hashCode() {
- return Objects.hash(name, dataTTL, childSize);
+ return Objects.hash(name, dataTTL, childSize, alignedTimeseriesIndex);
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java b/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
index b5906a8..b88a09f 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
@@ -78,141 +78,7 @@
import org.apache.iotdb.db.qp.logical.sys.StopTriggerOperator;
import org.apache.iotdb.db.qp.logical.sys.TracingOperator;
import org.apache.iotdb.db.qp.physical.crud.GroupByTimePlan;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AggregationCallContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AggregationElementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AliasClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AlignByDeviceClauseOrDisableAlignInSpecialLimitContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AlignByDeviceStatementOrDisableAlignInSpecialClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AlterClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AlterTimeseriesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AlterUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AndExpressionContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AsClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AsElementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AttributeClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.AttributeClausesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.BuiltInFunctionCallContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ClearcacheContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ConstantContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CountDevicesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CountNodesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CountStorageGroupContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CountTimeseriesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CreateFunctionContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CreateIndexContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CreateRoleContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CreateSnapshotContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CreateTimeseriesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CreateTriggerContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.CreateUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DateExpressionContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DeletePartitionContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DeleteStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DeleteStorageGroupContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DeleteTimeseriesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DropFunctionContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DropIndexContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DropRoleContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DropTriggerContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.DropUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FillClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FillStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FlushContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FromClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FullMergeContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FullPathContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FunctionAsClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.FunctionAsElementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GrantRoleContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GrantRoleToUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GrantUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GrantWatermarkEmbeddingContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GroupByFillClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GroupByFillStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GroupByLevelClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GroupByLevelStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GroupByTimeClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.GroupByTimeStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.InClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.IndexPredicateClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.IndexWithClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.InsertColumnsSpecContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.InsertStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.InsertValuesSpecContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.KillQueryContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.LastClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.LastElementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.LimitClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.LimitStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListAllRoleOfUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListAllUserOfRoleContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListPrivilegesRoleContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListPrivilegesUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListRoleContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListRolePrivilegesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ListUserPrivilegesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.LoadConfigurationStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.LoadFilesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.LoadStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.MergeContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.MoveFileContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.NodeNameContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.NodeNameWithoutStarContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.OffsetClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.OrExpressionContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.OrderByTimeClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.OrderByTimeStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.PredicateContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.PrefixPathContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.PrivilegesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.PropertyContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.PropertyValueContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.RemoveFileContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.RevokeRoleContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.RevokeRoleFromUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.RevokeUserContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.RevokeWatermarkEmbeddingContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.RootOrIdContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SelectStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SequenceClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SetStorageGroupContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SetTTLStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowAllTTLStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowChildNodesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowChildPathsContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowDevicesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowFlushTaskInfoContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowFunctionsContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowMergeStatusContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowQueryProcesslistContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowStorageGroupContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowTTLStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowTimeseriesContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowTriggersContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowVersionContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowWhereClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SingleStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SlimitClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SlimitStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SoffsetClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SpecialLimitStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.StartTriggerContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.StopTriggerContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.StringLiteralContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.SuffixPathContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TableCallContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TableElementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TagClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TimeIntervalContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TracingOffContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TracingOnContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TriggerAttributeContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.TypeClauseContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.UdfAttributeContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.UdfCallContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.UnsetTTLStatementContext;
-import org.apache.iotdb.db.qp.sql.SqlBaseParser.WhereClauseContext;
+import org.apache.iotdb.db.qp.sql.SqlBaseParser.*;
import org.apache.iotdb.db.qp.utils.DatetimeUtils;
import org.apache.iotdb.db.query.executor.fill.IFill;
import org.apache.iotdb.db.query.executor.fill.LinearFill;
@@ -1880,10 +1746,10 @@
}
private void parseInsertColumnSpec(InsertColumnsSpecContext ctx, InsertOperator insertOp) {
- List<NodeNameWithoutStarContext> nodeNamesWithoutStar = ctx.nodeNameWithoutStar();
+ List<MeasurementNameContext> measurementNames = ctx.measurementName();
List<String> measurementList = new ArrayList<>();
- for (NodeNameWithoutStarContext nodeNameWithoutStar : nodeNamesWithoutStar) {
- String measurement = nodeNameWithoutStar.getText();
+ for (MeasurementNameContext measurementName : measurementNames) {
+ String measurement = measurementName.getText();
measurementList.add(measurement);
}
insertOp.setMeasurementList(measurementList.toArray(new String[0]));
@@ -1898,9 +1764,11 @@
}
insertOp.setTime(timestamp);
List<String> valueList = new ArrayList<>();
- List<ConstantContext> values = ctx.constant();
- for (ConstantContext value : values) {
- valueList.add(value.getText());
+ List<MeasurementValueContext> values = ctx.measurementValue();
+ for (MeasurementValueContext value : values) {
+ for (ConstantContext counstant : value.constant()) {
+ valueList.add(counstant.getText());
+ }
}
insertOp.setValueList(valueList.toArray(new String[0]));
}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
index 66181d7..7b8c0ac 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
@@ -232,13 +232,20 @@
case INSERT:
InsertOperator insert = (InsertOperator) operator;
paths = insert.getSelectedPaths();
- if (insert.getMeasurementList().length != insert.getValueList().length) {
+ int measurementsNum = 0;
+ for (String measurement : insert.getMeasurementList()) {
+ if (measurement.startsWith("(") && measurement.endsWith(")")) {
+ measurementsNum += measurement.replace("(", "").replace(")", "").split(",").length;
+ } else {
+ measurementsNum++;
+ }
+ }
+ if (measurementsNum != insert.getValueList().length) {
throw new SQLParserException(
String.format(
"the measurementList's size %d is not consistent with the valueList's size %d",
- insert.getMeasurementList().length, insert.getValueList().length));
+ measurementsNum, insert.getValueList().length));
}
-
return new InsertRowPlan(
paths.get(0), insert.getTime(), insert.getMeasurementList(), insert.getValueList());
case MERGE:
@@ -849,6 +856,7 @@
columnForReaderSet.add(column);
}
}
+ ((LastQueryPlan) queryPlan).transformPaths(IoTDB.metaManager);
return;
}
@@ -903,10 +911,37 @@
columnForDisplaySet.add(columnForDisplay);
}
}
-
if (queryPlan instanceof UDTFPlan) {
((UDTFPlan) queryPlan).setPathNameToReaderIndex(pathNameToReaderIndex);
+ return;
}
+
+ if (!rawDataQueryPlan.isRawQuery()) {
+ rawDataQueryPlan.transformPaths(IoTDB.metaManager);
+ } else {
+ // support vector
+ List<PartialPath> deduplicatedPaths = rawDataQueryPlan.getDeduplicatedPaths();
+ Pair<List<PartialPath>, Map<String, Integer>> pair = getSeriesSchema(deduplicatedPaths);
+
+ List<PartialPath> vectorizedDeduplicatedPaths = pair.left;
+ List<TSDataType> vectorizedDeduplicatedDataTypes =
+ new ArrayList<>(getSeriesTypes(vectorizedDeduplicatedPaths));
+ rawDataQueryPlan.setDeduplicatedVectorPaths(vectorizedDeduplicatedPaths);
+ rawDataQueryPlan.setDeduplicatedVectorDataTypes(vectorizedDeduplicatedDataTypes);
+
+ Map<String, Integer> columnForDisplayToQueryDataSetIndex = pair.right;
+ Map<String, Integer> pathToIndex = new HashMap<>();
+ for (String columnForDisplay : columnForDisplaySet) {
+ pathToIndex.put(
+ columnForDisplay, columnForDisplayToQueryDataSetIndex.get(columnForDisplay));
+ }
+ queryPlan.setVectorPathToIndex(pathToIndex);
+ }
+ }
+
+ protected Pair<List<PartialPath>, Map<String, Integer>> getSeriesSchema(List<PartialPath> paths)
+ throws MetadataException {
+ return IoTDB.metaManager.getSeriesSchemas(paths);
}
private List<String> slimitTrimColumn(List<String> columnList, int seriesLimit, int seriesOffset)
diff --git a/server/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java b/server/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
index 326e71f..2a48f4a 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
@@ -22,7 +22,7 @@
import org.apache.iotdb.db.engine.modification.Modification;
import org.apache.iotdb.db.engine.modification.ModificationFile;
import org.apache.iotdb.db.metadata.PartialPath;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import java.util.ArrayList;
import java.util.HashMap;
@@ -106,7 +106,7 @@
this.queryTimeLowerBound = queryTimeLowerBound;
}
- public boolean chunkNotSatisfy(ChunkMetadata chunkMetaData) {
+ public boolean chunkNotSatisfy(IChunkMetadata chunkMetaData) {
return chunkMetaData.getEndTime() < queryTimeLowerBound;
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/AlignByDeviceDataSet.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/AlignByDeviceDataSet.java
index b37a16e..38e1821 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/AlignByDeviceDataSet.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/AlignByDeviceDataSet.java
@@ -24,6 +24,7 @@
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.metadata.PartialPath;
import org.apache.iotdb.db.metadata.mnode.MNode;
+import org.apache.iotdb.db.metadata.template.Template;
import org.apache.iotdb.db.qp.physical.crud.AggregationPlan;
import org.apache.iotdb.db.qp.physical.crud.AlignByDevicePlan;
import org.apache.iotdb.db.qp.physical.crud.AlignByDevicePlan.MeasurementType;
@@ -45,6 +46,7 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@@ -168,28 +170,32 @@
try {
switch (dataSetType) {
case GROUPBYTIME:
- groupByTimePlan.setDeduplicatedPaths(executePaths);
+ groupByTimePlan.setDeduplicatedPathsAndUpdate(executePaths);
groupByTimePlan.setDeduplicatedDataTypes(tsDataTypes);
groupByTimePlan.setDeduplicatedAggregations(executeAggregations);
groupByTimePlan.setExpression(expression);
+ groupByTimePlan.transformPaths(IoTDB.metaManager);
currentDataSet = queryRouter.groupBy(groupByTimePlan, context);
break;
case AGGREGATE:
- aggregationPlan.setDeduplicatedPaths(executePaths);
+ aggregationPlan.setDeduplicatedPathsAndUpdate(executePaths);
aggregationPlan.setDeduplicatedAggregations(executeAggregations);
aggregationPlan.setDeduplicatedDataTypes(tsDataTypes);
aggregationPlan.setExpression(expression);
+ aggregationPlan.transformPaths(IoTDB.metaManager);
currentDataSet = queryRouter.aggregate(aggregationPlan, context);
break;
case FILL:
fillQueryPlan.setDeduplicatedDataTypes(tsDataTypes);
- fillQueryPlan.setDeduplicatedPaths(executePaths);
+ fillQueryPlan.setDeduplicatedPathsAndUpdate(executePaths);
+ fillQueryPlan.transformPaths(IoTDB.metaManager);
currentDataSet = queryRouter.fill(fillQueryPlan, context);
break;
case QUERY:
- rawDataQueryPlan.setDeduplicatedPaths(executePaths);
+ rawDataQueryPlan.setDeduplicatedPathsAndUpdate(executePaths);
rawDataQueryPlan.setDeduplicatedDataTypes(tsDataTypes);
rawDataQueryPlan.setExpression(expression);
+ rawDataQueryPlan.transformPaths(IoTDB.metaManager);
currentDataSet = queryRouter.rawDataQuery(rawDataQueryPlan, context);
break;
default:
@@ -197,7 +203,8 @@
}
} catch (QueryProcessException
| QueryFilterOptimizationException
- | StorageEngineException e) {
+ | StorageEngineException
+ | MetadataException e) {
throw new IOException(e);
}
@@ -220,7 +227,13 @@
protected Set<String> getDeviceMeasurements(PartialPath device) throws IOException {
try {
MNode deviceNode = IoTDB.metaManager.getNodeByPath(device);
- return deviceNode.getChildren().keySet();
+ Set<String> res = new HashSet<>(deviceNode.getChildren().keySet());
+ Template template = deviceNode.getUpperTemplate();
+ if (template != null) {
+ res.addAll(template.getSchemaMap().keySet());
+ }
+
+ return res;
} catch (MetadataException e) {
throw new IOException("Cannot get node from " + device, e);
}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
index 0a02f94..88c78ee 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
@@ -24,6 +24,7 @@
import org.apache.iotdb.tsfile.read.common.RowRecord;
import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
import org.apache.iotdb.tsfile.read.query.timegenerator.TimeGenerator;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
import java.io.IOException;
import java.util.ArrayList;
@@ -121,7 +122,12 @@
rowRecords[j].addField(null);
} else {
hasField[j] = true;
- rowRecords[j].addField(results[j], dataTypes.get(i));
+ if (dataTypes.get(i) == TSDataType.VECTOR) {
+ TsPrimitiveType[] result = (TsPrimitiveType[]) results[j];
+ rowRecords[j].addField(result[0].getValue(), result[0].getDataType());
+ } else {
+ rowRecords[j].addField(results[j], dataTypes.get(i));
+ }
}
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithoutValueFilter.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithoutValueFilter.java
index c46fe1d..dbcea9d 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithoutValueFilter.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithoutValueFilter.java
@@ -21,6 +21,7 @@
import org.apache.iotdb.db.concurrent.WrappedRunnable;
import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.VectorPartialPath;
import org.apache.iotdb.db.query.control.QueryTimeManager;
import org.apache.iotdb.db.query.pool.QueryTaskPoolManager;
import org.apache.iotdb.db.query.reader.series.ManagedSeriesReader;
@@ -37,6 +38,7 @@
import org.apache.iotdb.tsfile.utils.BytesUtils;
import org.apache.iotdb.tsfile.utils.PublicBAOS;
import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -144,6 +146,8 @@
protected BatchData[] cachedBatchDataArray;
+ private int bufferNum;
+
// capacity for blocking queue
private static final int BLOCKING_QUEUE_CAPACITY = 5;
@@ -177,6 +181,14 @@
}
cachedBatchDataArray = new BatchData[readers.size()];
noMoreDataInQueueArray = new boolean[readers.size()];
+ bufferNum = 0;
+ for (PartialPath path : paths) {
+ if (path instanceof VectorPartialPath) {
+ bufferNum += ((VectorPartialPath) path).getSubSensorsPathList().size();
+ } else {
+ bufferNum += 1;
+ }
+ }
init();
}
@@ -224,16 +236,17 @@
TSQueryDataSet tsQueryDataSet = new TSQueryDataSet();
PublicBAOS timeBAOS = new PublicBAOS();
- PublicBAOS[] valueBAOSList = new PublicBAOS[seriesNum];
- PublicBAOS[] bitmapBAOSList = new PublicBAOS[seriesNum];
- for (int seriesIndex = 0; seriesIndex < seriesNum; seriesIndex++) {
- valueBAOSList[seriesIndex] = new PublicBAOS();
- bitmapBAOSList[seriesIndex] = new PublicBAOS();
+ PublicBAOS[] valueBAOSList = new PublicBAOS[bufferNum];
+ PublicBAOS[] bitmapBAOSList = new PublicBAOS[bufferNum];
+
+ for (int bufferIndex = 0; bufferIndex < bufferNum; bufferIndex++) {
+ valueBAOSList[bufferIndex] = new PublicBAOS();
+ bitmapBAOSList[bufferIndex] = new PublicBAOS();
}
// used to record a bitmap for every 8 row records
- int[] currentBitmapList = new int[seriesNum];
+ int[] currentBitmapList = new int[bufferNum];
int rowCount = 0;
while (rowCount < fetchSize) {
@@ -247,56 +260,127 @@
timeBAOS.write(BytesUtils.longToBytes(minTime));
}
- for (int seriesIndex = 0; seriesIndex < seriesNum; seriesIndex++) {
+ for (int seriesIndex = 0, bufferIndex = 0; seriesIndex < seriesNum; seriesIndex++) {
if (cachedBatchDataArray[seriesIndex] == null
|| !cachedBatchDataArray[seriesIndex].hasCurrent()
|| cachedBatchDataArray[seriesIndex].currentTime() != minTime) {
// current batch is empty or does not have value at minTime
if (rowOffset == 0) {
- currentBitmapList[seriesIndex] = (currentBitmapList[seriesIndex] << 1);
+ if (paths.get(seriesIndex) instanceof VectorPartialPath) {
+ for (int i = 0;
+ i < ((VectorPartialPath) paths.get(seriesIndex)).getSubSensorsPathList().size();
+ i++) {
+ currentBitmapList[bufferIndex] = (currentBitmapList[bufferIndex] << 1);
+ bufferIndex++;
+ }
+ } else {
+ currentBitmapList[bufferIndex] = (currentBitmapList[bufferIndex] << 1);
+ bufferIndex++;
+ }
}
} else {
// current batch has value at minTime, consume current value
if (rowOffset == 0) {
- currentBitmapList[seriesIndex] = (currentBitmapList[seriesIndex] << 1) | FLAG;
TSDataType type = cachedBatchDataArray[seriesIndex].getDataType();
switch (type) {
case INT32:
+ currentBitmapList[bufferIndex] = (currentBitmapList[bufferIndex] << 1) | FLAG;
int intValue = cachedBatchDataArray[seriesIndex].getInt();
if (encoder != null && encoder.needEncode(minTime)) {
intValue = encoder.encodeInt(intValue, minTime);
}
ReadWriteIOUtils.write(intValue, valueBAOSList[seriesIndex]);
+ bufferIndex++;
break;
case INT64:
+ currentBitmapList[bufferIndex] = (currentBitmapList[bufferIndex] << 1) | FLAG;
long longValue = cachedBatchDataArray[seriesIndex].getLong();
if (encoder != null && encoder.needEncode(minTime)) {
longValue = encoder.encodeLong(longValue, minTime);
}
ReadWriteIOUtils.write(longValue, valueBAOSList[seriesIndex]);
+ bufferIndex++;
break;
case FLOAT:
+ currentBitmapList[bufferIndex] = (currentBitmapList[bufferIndex] << 1) | FLAG;
float floatValue = cachedBatchDataArray[seriesIndex].getFloat();
if (encoder != null && encoder.needEncode(minTime)) {
floatValue = encoder.encodeFloat(floatValue, minTime);
}
ReadWriteIOUtils.write(floatValue, valueBAOSList[seriesIndex]);
+ bufferIndex++;
break;
case DOUBLE:
+ currentBitmapList[bufferIndex] = (currentBitmapList[bufferIndex] << 1) | FLAG;
double doubleValue = cachedBatchDataArray[seriesIndex].getDouble();
if (encoder != null && encoder.needEncode(minTime)) {
doubleValue = encoder.encodeDouble(doubleValue, minTime);
}
ReadWriteIOUtils.write(doubleValue, valueBAOSList[seriesIndex]);
+ bufferIndex++;
break;
case BOOLEAN:
+ currentBitmapList[bufferIndex] = (currentBitmapList[bufferIndex] << 1) | FLAG;
ReadWriteIOUtils.write(
cachedBatchDataArray[seriesIndex].getBoolean(), valueBAOSList[seriesIndex]);
+ bufferIndex++;
break;
case TEXT:
+ currentBitmapList[bufferIndex] = (currentBitmapList[bufferIndex] << 1) | FLAG;
ReadWriteIOUtils.write(
cachedBatchDataArray[seriesIndex].getBinary(), valueBAOSList[seriesIndex]);
+ bufferIndex++;
+ break;
+ case VECTOR:
+ for (TsPrimitiveType primitiveVal : cachedBatchDataArray[seriesIndex].getVector()) {
+ if (primitiveVal == null) {
+ currentBitmapList[bufferIndex] = (currentBitmapList[bufferIndex] << 1);
+ bufferIndex++;
+ continue;
+ }
+ currentBitmapList[bufferIndex] = (currentBitmapList[bufferIndex] << 1) | FLAG;
+ switch (primitiveVal.getDataType()) {
+ case INT32:
+ int intVal = primitiveVal.getInt();
+ if (encoder != null && encoder.needEncode(minTime)) {
+ intVal = encoder.encodeInt(intVal, minTime);
+ }
+ ReadWriteIOUtils.write(intVal, valueBAOSList[bufferIndex]);
+ break;
+ case INT64:
+ long longVal = primitiveVal.getLong();
+ if (encoder != null && encoder.needEncode(minTime)) {
+ longVal = encoder.encodeLong(longVal, minTime);
+ }
+ ReadWriteIOUtils.write(longVal, valueBAOSList[bufferIndex]);
+ break;
+ case FLOAT:
+ float floatVal = primitiveVal.getFloat();
+ if (encoder != null && encoder.needEncode(minTime)) {
+ floatVal = encoder.encodeFloat(floatVal, minTime);
+ }
+ ReadWriteIOUtils.write(floatVal, valueBAOSList[bufferIndex]);
+ break;
+ case DOUBLE:
+ double doubleVal = primitiveVal.getDouble();
+ if (encoder != null && encoder.needEncode(minTime)) {
+ doubleVal = encoder.encodeDouble(doubleVal, minTime);
+ }
+ ReadWriteIOUtils.write(doubleVal, valueBAOSList[bufferIndex]);
+ break;
+ case BOOLEAN:
+ ReadWriteIOUtils.write(primitiveVal.getBoolean(), valueBAOSList[bufferIndex]);
+ break;
+ case TEXT:
+ ReadWriteIOUtils.write(primitiveVal.getBinary(), valueBAOSList[bufferIndex]);
+ break;
+ default:
+ throw new UnSupportedDataTypeException(
+ String.format("Data type %s is not supported.", type));
+ }
+ bufferIndex++;
+ }
break;
default:
throw new UnSupportedDataTypeException(
@@ -326,11 +410,11 @@
if (rowOffset == 0) {
rowCount++;
if (rowCount % 8 == 0) {
- for (int seriesIndex = 0; seriesIndex < seriesNum; seriesIndex++) {
+ for (int bufferIndex = 0; bufferIndex < bufferNum; bufferIndex++) {
ReadWriteIOUtils.write(
- (byte) currentBitmapList[seriesIndex], bitmapBAOSList[seriesIndex]);
+ (byte) currentBitmapList[bufferIndex], bitmapBAOSList[bufferIndex]);
// we should clear the bitmap every 8 row record
- currentBitmapList[seriesIndex] = 0;
+ currentBitmapList[bufferIndex] = 0;
}
}
if (rowLimit > 0) {
@@ -348,10 +432,10 @@
if (rowCount > 0) {
int remaining = rowCount % 8;
if (remaining != 0) {
- for (int seriesIndex = 0; seriesIndex < seriesNum; seriesIndex++) {
+ for (int bufferIndex = 0; bufferIndex < bufferNum; bufferIndex++) {
ReadWriteIOUtils.write(
- (byte) (currentBitmapList[seriesIndex] << (8 - remaining)),
- bitmapBAOSList[seriesIndex]);
+ (byte) (currentBitmapList[bufferIndex] << (8 - remaining)),
+ bitmapBAOSList[bufferIndex]);
}
}
}
@@ -365,13 +449,13 @@
List<ByteBuffer> valueBufferList = new ArrayList<>();
List<ByteBuffer> bitmapBufferList = new ArrayList<>();
- for (int seriesIndex = 0; seriesIndex < seriesNum; seriesIndex++) {
+ for (int bufferIndex = 0; bufferIndex < bufferNum; bufferIndex++) {
// add value buffer of current series
- putPBOSToBuffer(valueBAOSList, valueBufferList, seriesIndex);
+ putPBOSToBuffer(valueBAOSList, valueBufferList, bufferIndex);
// add bitmap buffer of current series
- putPBOSToBuffer(bitmapBAOSList, bitmapBufferList, seriesIndex);
+ putPBOSToBuffer(bitmapBAOSList, bitmapBufferList, bufferIndex);
}
// set value buffers and bitmap buffers
@@ -432,6 +516,7 @@
}
/** for spark/hadoop/hive integration and test */
+ @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
@Override
public RowRecord nextWithoutConstraint() throws IOException {
long minTime = timeHeap.pollFirst();
@@ -442,10 +527,28 @@
if (cachedBatchDataArray[seriesIndex] == null
|| !cachedBatchDataArray[seriesIndex].hasCurrent()
|| cachedBatchDataArray[seriesIndex].currentTime() != minTime) {
- record.addField(null);
+ if (paths.get(seriesIndex) instanceof VectorPartialPath) {
+ for (int i = 0;
+ i < ((VectorPartialPath) paths.get(seriesIndex)).getSubSensorsPathList().size();
+ i++) {
+ record.addField(null);
+ }
+ } else {
+ record.addField(null);
+ }
} else {
TSDataType dataType = dataTypes.get(seriesIndex);
- record.addField(cachedBatchDataArray[seriesIndex].currentValue(), dataType);
+ if (dataType == TSDataType.VECTOR) {
+ for (TsPrimitiveType primitiveVal : cachedBatchDataArray[seriesIndex].getVector()) {
+ if (primitiveVal == null) {
+ record.addField(null);
+ } else {
+ record.addField(primitiveVal.getValue(), primitiveVal.getDataType());
+ }
+ }
+ } else {
+ record.addField(cachedBatchDataArray[seriesIndex].currentValue(), dataType);
+ }
cacheNext(seriesIndex);
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java b/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java
index 287856e..bebe02b 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java
@@ -20,6 +20,7 @@
package org.apache.iotdb.db.query.executor;
import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.metadata.PartialPath;
import org.apache.iotdb.db.qp.physical.crud.AggregationPlan;
@@ -36,6 +37,7 @@
import org.apache.iotdb.db.query.dataset.groupby.GroupByWithValueFilterDataSet;
import org.apache.iotdb.db.query.dataset.groupby.GroupByWithoutValueFilterDataSet;
import org.apache.iotdb.db.query.executor.fill.IFill;
+import org.apache.iotdb.db.service.IoTDB;
import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.expression.ExpressionType;
@@ -89,8 +91,16 @@
if (optimizedExpression != null
&& optimizedExpression.getType() != ExpressionType.GLOBAL_TIME) {
+ try {
+ queryPlan.transformPaths(IoTDB.metaManager);
+ } catch (MetadataException e) {
+ throw new QueryProcessException(e);
+ }
return rawDataQueryExecutor.executeWithValueFilter(context);
}
+
+ // Currently, we only group the vector partial paths for raw query without value filter
+ queryPlan.transformToVector();
return rawDataQueryExecutor.executeWithoutValueFilter(context);
}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/fill/LastPointReader.java b/server/src/main/java/org/apache/iotdb/db/query/executor/fill/LastPointReader.java
index dada7da..e7eb64f 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/executor/fill/LastPointReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/executor/fill/LastPointReader.java
@@ -21,10 +21,14 @@
import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.VectorPartialPath;
import org.apache.iotdb.db.query.context.QueryContext;
import org.apache.iotdb.db.utils.FileLoaderUtils;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ITimeSeriesMetadata;
import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.file.metadata.VectorTimeSeriesMetadata;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
import org.apache.iotdb.tsfile.read.TimeValuePair;
@@ -35,6 +39,7 @@
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
import java.util.PriorityQueue;
import java.util.Set;
@@ -102,9 +107,24 @@
TimeValuePair lastPoint = new TimeValuePair(Long.MIN_VALUE, null);
for (int index = seqFileResource.size() - 1; index >= 0; index--) {
TsFileResource resource = seqFileResource.get(index);
- TimeseriesMetadata timeseriesMetadata =
- FileLoaderUtils.loadTimeSeriesMetadata(
- resource, seriesPath, context, timeFilter, deviceMeasurements);
+ ITimeSeriesMetadata timeseriesMetadata;
+ if (seriesPath instanceof VectorPartialPath) {
+ timeseriesMetadata =
+ new VectorTimeSeriesMetadata(
+ FileLoaderUtils.loadTimeSeriesMetadata(
+ resource, seriesPath, context, timeFilter, deviceMeasurements),
+ Collections.singletonList(
+ FileLoaderUtils.loadTimeSeriesMetadata(
+ resource,
+ ((VectorPartialPath) seriesPath).getSubSensorsPathList().get(0),
+ context,
+ timeFilter,
+ deviceMeasurements)));
+ } else {
+ timeseriesMetadata =
+ FileLoaderUtils.loadTimeSeriesMetadata(
+ resource, seriesPath, context, timeFilter, deviceMeasurements);
+ }
if (timeseriesMetadata != null) {
if (!timeseriesMetadata.isModified()
&& endtimeContainedByTimeFilter(timeseriesMetadata.getStatistics())) {
@@ -113,9 +133,9 @@
timeseriesMetadata.getStatistics().getLastValue(),
dataType);
} else {
- List<ChunkMetadata> seqChunkMetadataList = timeseriesMetadata.loadChunkMetadataList();
+ List<IChunkMetadata> seqChunkMetadataList = timeseriesMetadata.loadChunkMetadataList();
for (int i = seqChunkMetadataList.size() - 1; i >= 0; i--) {
- lastPoint = getChunkLastPoint(seqChunkMetadataList.get(i));
+ lastPoint = getChunkLastPoint((ChunkMetadata) seqChunkMetadataList.get(i));
// last point of this sequence chunk is valid, quit the loop
if (lastPoint.getValue() != null) {
return lastPoint;
@@ -223,7 +243,9 @@
});
for (TimeseriesMetadata timeseriesMetadata : unseqTimeseriesMetadataList) {
if (timeseriesMetadata != null) {
- chunkMetadataList.addAll(timeseriesMetadata.loadChunkMetadataList());
+ for (IChunkMetadata chunkMetadata : timeseriesMetadata.loadChunkMetadataList()) {
+ chunkMetadataList.add((ChunkMetadata) chunkMetadata);
+ }
}
}
return chunkMetadataList;
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/MemPageReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/MemPageReader.java
index c05cb76..c4d8a09 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/MemPageReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/MemPageReader.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.query.reader.chunk;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
import org.apache.iotdb.tsfile.read.TimeValuePair;
import org.apache.iotdb.tsfile.read.common.BatchData;
@@ -33,11 +33,11 @@
public class MemPageReader implements IPageReader {
private final IPointReader timeValuePairIterator;
- private final ChunkMetadata chunkMetadata;
+ private final IChunkMetadata chunkMetadata;
private Filter valueFilter;
public MemPageReader(
- IPointReader timeValuePairIterator, ChunkMetadata chunkMetadata, Filter filter) {
+ IPointReader timeValuePairIterator, IChunkMetadata chunkMetadata, Filter filter) {
this.timeValuePairIterator = timeValuePairIterator;
this.chunkMetadata = chunkMetadata;
this.valueFilter = filter;
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/DiskChunkMetadataLoader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/DiskChunkMetadataLoader.java
index cdaf128..2d457ac 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/DiskChunkMetadataLoader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/DiskChunkMetadataLoader.java
@@ -24,8 +24,8 @@
import org.apache.iotdb.db.query.context.QueryContext;
import org.apache.iotdb.db.query.reader.chunk.DiskChunkLoader;
import org.apache.iotdb.db.utils.QueryUtils;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
-import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ITimeSeriesMetadata;
import org.apache.iotdb.tsfile.read.controller.IChunkMetadataLoader;
import org.apache.iotdb.tsfile.read.filter.basic.Filter;
@@ -53,8 +53,8 @@
}
@Override
- public List<ChunkMetadata> loadChunkMetadataList(TimeseriesMetadata timeseriesMetadata) {
- List<ChunkMetadata> chunkMetadataList = timeseriesMetadata.getChunkMetadataList();
+ public List<IChunkMetadata> loadChunkMetadataList(ITimeSeriesMetadata timeseriesMetadata) {
+ List<IChunkMetadata> chunkMetadataList = timeseriesMetadata.getChunkMetadataList();
setDiskChunkLoader(chunkMetadataList, resource, seriesPath, context);
@@ -69,7 +69,7 @@
|| chunkMetaData.getStartTime() > chunkMetaData.getEndTime());
// For chunkMetadata from old TsFile, do not set version
- for (ChunkMetadata metadata : chunkMetadataList) {
+ for (IChunkMetadata metadata : chunkMetadataList) {
if (!metadata.isFromOldTsFile()) {
metadata.setVersion(resource.getVersion());
}
@@ -83,8 +83,13 @@
return chunkMetadataList;
}
+ @Override
+ public boolean isMemChunkMetadataLoader() {
+ return false;
+ }
+
public static void setDiskChunkLoader(
- List<ChunkMetadata> chunkMetadataList,
+ List<IChunkMetadata> chunkMetadataList,
TsFileResource resource,
PartialPath seriesPath,
QueryContext context) {
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/MemChunkMetadataLoader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/MemChunkMetadataLoader.java
index 0d7b1d5..cf022e3 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/MemChunkMetadataLoader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/MemChunkMetadataLoader.java
@@ -22,8 +22,8 @@
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.metadata.PartialPath;
import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
-import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ITimeSeriesMetadata;
import org.apache.iotdb.tsfile.read.controller.IChunkMetadataLoader;
import org.apache.iotdb.tsfile.read.filter.basic.Filter;
@@ -45,8 +45,8 @@
}
@Override
- public List<ChunkMetadata> loadChunkMetadataList(TimeseriesMetadata timeseriesMetadata) {
- List<ChunkMetadata> chunkMetadataList = resource.getChunkMetadataList();
+ public List<IChunkMetadata> loadChunkMetadataList(ITimeSeriesMetadata timeseriesMetadata) {
+ List<IChunkMetadata> chunkMetadataList = resource.getChunkMetadataList();
DiskChunkMetadataLoader.setDiskChunkLoader(chunkMetadataList, resource, seriesPath, context);
@@ -68,9 +68,14 @@
chunkMetaData.getStartTime(), chunkMetaData.getEndTime()))
|| chunkMetaData.getStartTime() > chunkMetaData.getEndTime());
- for (ChunkMetadata metadata : chunkMetadataList) {
+ for (IChunkMetadata metadata : chunkMetadataList) {
metadata.setVersion(resource.getVersion());
}
return chunkMetadataList;
}
+
+ @Override
+ public boolean isMemChunkMetadataLoader() {
+ return true;
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReader.java
index 04cb907..a76327d 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReader.java
@@ -45,7 +45,7 @@
TsFileFilter fileFilter,
boolean ascending) {
this.seriesReader =
- new SeriesReader(
+ SeriesReaderFactory.createSeriesReader(
seriesPath,
allSensors,
dataType,
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesRawDataBatchReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesRawDataBatchReader.java
index 8698955..d53db04 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesRawDataBatchReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesRawDataBatchReader.java
@@ -58,7 +58,7 @@
TsFileFilter fileFilter,
boolean ascending) {
this.seriesReader =
- new SeriesReader(
+ SeriesReaderFactory.createSeriesReader(
seriesPath,
allSensors,
dataType,
@@ -84,7 +84,7 @@
Set<String> allSensors = new HashSet<>();
allSensors.add(seriesPath.getMeasurement());
this.seriesReader =
- new SeriesReader(
+ SeriesReaderFactory.createSeriesReader(
seriesPath,
allSensors,
dataType,
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
index 01cb4f1..041fa76 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
@@ -32,7 +32,8 @@
import org.apache.iotdb.db.utils.FileLoaderUtils;
import org.apache.iotdb.db.utils.QueryUtils;
import org.apache.iotdb.db.utils.TestOnly;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ITimeSeriesMetadata;
import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
@@ -57,14 +58,14 @@
public class SeriesReader {
// inner class of SeriesReader for order purpose
- private TimeOrderUtils orderUtils;
+ protected TimeOrderUtils orderUtils;
- private final PartialPath seriesPath;
+ protected final PartialPath seriesPath;
// all the sensors in this device;
- private final Set<String> allSensors;
- private final TSDataType dataType;
- private final QueryContext context;
+ protected final Set<String> allSensors;
+ protected final TSDataType dataType;
+ protected final QueryContext context;
/*
* There is at most one is not null between timeFilter and valueFilter
@@ -73,44 +74,44 @@
*
* valueFilter is pushed down to non-overlapped page only
*/
- private final Filter timeFilter;
- private final Filter valueFilter;
+ protected final Filter timeFilter;
+ protected final Filter valueFilter;
/*
* file cache
*/
- private final List<TsFileResource> seqFileResource;
- private final List<TsFileResource> unseqFileResource;
+ protected final List<TsFileResource> seqFileResource;
+ protected final List<TsFileResource> unseqFileResource;
/*
* TimeSeriesMetadata cache
*/
- private TimeseriesMetadata firstTimeSeriesMetadata;
- private final List<TimeseriesMetadata> seqTimeSeriesMetadata = new LinkedList<>();
- private final PriorityQueue<TimeseriesMetadata> unSeqTimeSeriesMetadata;
+ protected ITimeSeriesMetadata firstTimeSeriesMetadata;
+ protected final List<ITimeSeriesMetadata> seqTimeSeriesMetadata = new LinkedList<>();
+ protected final PriorityQueue<ITimeSeriesMetadata> unSeqTimeSeriesMetadata;
/*
* chunk cache
*/
- private ChunkMetadata firstChunkMetadata;
- private final PriorityQueue<ChunkMetadata> cachedChunkMetadata;
+ protected IChunkMetadata firstChunkMetadata;
+ protected final PriorityQueue<IChunkMetadata> cachedChunkMetadata;
/*
* page cache
*/
- private VersionPageReader firstPageReader;
- private final List<VersionPageReader> seqPageReaders = new LinkedList<>();
- private final PriorityQueue<VersionPageReader> unSeqPageReaders;
+ protected VersionPageReader firstPageReader;
+ protected final List<VersionPageReader> seqPageReaders = new LinkedList<>();
+ protected final PriorityQueue<VersionPageReader> unSeqPageReaders;
/*
* point cache
*/
- private final PriorityMergeReader mergeReader;
+ protected final PriorityMergeReader mergeReader;
/*
* result cache
*/
- private boolean hasCachedNextOverlappedPage;
- private BatchData cachedBatchData;
+ protected boolean hasCachedNextOverlappedPage;
+ protected BatchData cachedBatchData;
public SeriesReader(
PartialPath seriesPath,
@@ -328,9 +329,9 @@
}
}
- private void unpackOneTimeSeriesMetadata(TimeseriesMetadata timeSeriesMetadata)
+ protected void unpackOneTimeSeriesMetadata(ITimeSeriesMetadata timeSeriesMetadata)
throws IOException {
- List<ChunkMetadata> chunkMetadataList =
+ List<IChunkMetadata> chunkMetadataList =
FileLoaderUtils.loadChunkMetadataList(timeSeriesMetadata);
chunkMetadataList.forEach(chunkMetadata -> chunkMetadata.setSeq(timeSeriesMetadata.isSeq()));
@@ -508,7 +509,7 @@
}
}
- private void unpackOneChunkMetaData(ChunkMetadata chunkMetaData) throws IOException {
+ private void unpackOneChunkMetaData(IChunkMetadata chunkMetaData) throws IOException {
FileLoaderUtils.loadPageReaderList(chunkMetaData, timeFilter)
.forEach(
pageReader -> {
@@ -737,9 +738,12 @@
*/
timeValuePair = mergeReader.nextTimeValuePair();
+ Object valueForFilter =
+ timeValuePair.getValue().getDataType() == TSDataType.VECTOR
+ ? timeValuePair.getValue().getVector()[0].getValue()
+ : timeValuePair.getValue().getValue();
if (valueFilter == null
- || valueFilter.satisfy(
- timeValuePair.getTimestamp(), timeValuePair.getValue().getValue())) {
+ || valueFilter.satisfy(timeValuePair.getTimestamp(), valueForFilter)) {
cachedBatchData.putAnObject(
timeValuePair.getTimestamp(), timeValuePair.getValue().getValue());
}
@@ -856,36 +860,19 @@
* approach is likely to be ubiquitous, but it keeps the system running smoothly
*/
@SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
- private void tryToUnpackAllOverlappedFilesToTimeSeriesMetadata() throws IOException {
+ protected void tryToUnpackAllOverlappedFilesToTimeSeriesMetadata() throws IOException {
/*
* Fill sequence TimeSeriesMetadata List until it is not empty
*/
while (seqTimeSeriesMetadata.isEmpty() && !seqFileResource.isEmpty()) {
- TimeseriesMetadata timeseriesMetadata =
- FileLoaderUtils.loadTimeSeriesMetadata(
- orderUtils.getNextSeqFileResource(seqFileResource, true),
- seriesPath,
- context,
- getAnyFilter(),
- allSensors);
- if (timeseriesMetadata != null) {
- timeseriesMetadata.setSeq(true);
- seqTimeSeriesMetadata.add(timeseriesMetadata);
- }
+ unpackSeqTsFileResource();
}
/*
* Fill unSequence TimeSeriesMetadata Priority Queue until it is not empty
*/
while (unSeqTimeSeriesMetadata.isEmpty() && !unseqFileResource.isEmpty()) {
- TimeseriesMetadata timeseriesMetadata =
- FileLoaderUtils.loadTimeSeriesMetadata(
- unseqFileResource.remove(0), seriesPath, context, getAnyFilter(), allSensors);
- if (timeseriesMetadata != null) {
- timeseriesMetadata.setModified(true);
- timeseriesMetadata.setSeq(false);
- unSeqTimeSeriesMetadata.add(timeseriesMetadata);
- }
+ unpackUnseqTsFileResource();
}
/*
@@ -934,37 +921,45 @@
}
}
- private void unpackAllOverlappedTsFilesToTimeSeriesMetadata(long endpointTime)
+ protected void unpackAllOverlappedTsFilesToTimeSeriesMetadata(long endpointTime)
throws IOException {
while (!unseqFileResource.isEmpty()
&& orderUtils.isOverlapped(endpointTime, unseqFileResource.get(0))) {
- TimeseriesMetadata timeseriesMetadata =
- FileLoaderUtils.loadTimeSeriesMetadata(
- unseqFileResource.remove(0), seriesPath, context, getAnyFilter(), allSensors);
- if (timeseriesMetadata != null) {
- timeseriesMetadata.setModified(true);
- timeseriesMetadata.setSeq(false);
- unSeqTimeSeriesMetadata.add(timeseriesMetadata);
- }
+ unpackUnseqTsFileResource();
}
while (!seqFileResource.isEmpty()
&& orderUtils.isOverlapped(
endpointTime, orderUtils.getNextSeqFileResource(seqFileResource, false))) {
- TimeseriesMetadata timeseriesMetadata =
- FileLoaderUtils.loadTimeSeriesMetadata(
- orderUtils.getNextSeqFileResource(seqFileResource, true),
- seriesPath,
- context,
- getAnyFilter(),
- allSensors);
- if (timeseriesMetadata != null) {
- timeseriesMetadata.setSeq(true);
- seqTimeSeriesMetadata.add(timeseriesMetadata);
- }
+ unpackSeqTsFileResource();
}
}
- private Filter getAnyFilter() {
+ protected void unpackSeqTsFileResource() throws IOException {
+ TimeseriesMetadata timeseriesMetadata =
+ FileLoaderUtils.loadTimeSeriesMetadata(
+ orderUtils.getNextSeqFileResource(seqFileResource, true),
+ seriesPath,
+ context,
+ getAnyFilter(),
+ allSensors);
+ if (timeseriesMetadata != null) {
+ timeseriesMetadata.setSeq(true);
+ seqTimeSeriesMetadata.add(timeseriesMetadata);
+ }
+ }
+
+ protected void unpackUnseqTsFileResource() throws IOException {
+ TimeseriesMetadata timeseriesMetadata =
+ FileLoaderUtils.loadTimeSeriesMetadata(
+ unseqFileResource.remove(0), seriesPath, context, getAnyFilter(), allSensors);
+ if (timeseriesMetadata != null) {
+ timeseriesMetadata.setModified(true);
+ timeseriesMetadata.setSeq(false);
+ unSeqTimeSeriesMetadata.add(timeseriesMetadata);
+ }
+ }
+
+ protected Filter getAnyFilter() {
return timeFilter != null ? timeFilter : valueFilter;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java
index 69d3248..543c706 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java
@@ -47,8 +47,8 @@
boolean ascending) {
UnaryFilter timeFilter =
ascending ? TimeFilter.gtEq(Long.MIN_VALUE) : TimeFilter.ltEq(Long.MAX_VALUE);
- seriesReader =
- new SeriesReader(
+ this.seriesReader =
+ SeriesReaderFactory.createSeriesReader(
seriesPath,
allSensors,
dataType,
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderFactory.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderFactory.java
new file mode 100644
index 0000000..577e021
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderFactory.java
@@ -0,0 +1,94 @@
+/*
+ * 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.query.reader.series;
+
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.VectorPartialPath;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.filter.TsFileFilter;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+
+import java.util.List;
+import java.util.Set;
+
+public class SeriesReaderFactory {
+
+ private SeriesReaderFactory() {}
+
+ public static SeriesReader createSeriesReader(
+ PartialPath seriesPath,
+ Set<String> allSensors,
+ TSDataType dataType,
+ QueryContext context,
+ QueryDataSource dataSource,
+ Filter timeFilter,
+ Filter valueFilter,
+ TsFileFilter fileFilter,
+ boolean ascending) {
+ if (seriesPath instanceof VectorPartialPath) {
+ return new VectorSeriesReader(
+ seriesPath,
+ allSensors,
+ dataType,
+ context,
+ dataSource,
+ timeFilter,
+ valueFilter,
+ fileFilter,
+ ascending);
+ } else {
+ return new SeriesReader(
+ seriesPath,
+ allSensors,
+ dataType,
+ context,
+ dataSource,
+ timeFilter,
+ valueFilter,
+ fileFilter,
+ ascending);
+ }
+ }
+
+ public static SeriesReader createSeriesReader(
+ PartialPath seriesPath,
+ Set<String> allSensors,
+ TSDataType dataType,
+ QueryContext context,
+ List<TsFileResource> seqFileResource,
+ List<TsFileResource> unseqFileResource,
+ Filter timeFilter,
+ Filter valueFilter,
+ boolean ascending) {
+ return new SeriesReader(
+ seriesPath,
+ allSensors,
+ dataType,
+ context,
+ seqFileResource,
+ unseqFileResource,
+ timeFilter,
+ valueFilter,
+ ascending);
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/VectorSeriesReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/VectorSeriesReader.java
new file mode 100644
index 0000000..6de0150
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/VectorSeriesReader.java
@@ -0,0 +1,144 @@
+/*
+ * 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.query.reader.series;
+
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.VectorPartialPath;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.filter.TsFileFilter;
+import org.apache.iotdb.db.utils.FileLoaderUtils;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.file.metadata.VectorTimeSeriesMetadata;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+public class VectorSeriesReader extends SeriesReader {
+
+ private final VectorPartialPath vectorPartialPath;
+
+ public VectorSeriesReader(
+ PartialPath seriesPath,
+ Set<String> allSensors,
+ TSDataType dataType,
+ QueryContext context,
+ QueryDataSource dataSource,
+ Filter timeFilter,
+ Filter valueFilter,
+ TsFileFilter fileFilter,
+ boolean ascending) {
+ super(
+ seriesPath,
+ allSensors,
+ dataType,
+ context,
+ dataSource,
+ timeFilter,
+ valueFilter,
+ fileFilter,
+ ascending);
+ this.allSensors.add(seriesPath.getMeasurement());
+ this.vectorPartialPath = (VectorPartialPath) seriesPath;
+ }
+
+ @TestOnly
+ VectorSeriesReader(
+ PartialPath seriesPath,
+ Set<String> allSensors,
+ TSDataType dataType,
+ QueryContext context,
+ List<TsFileResource> seqFileResource,
+ List<TsFileResource> unseqFileResource,
+ Filter timeFilter,
+ Filter valueFilter,
+ boolean ascending) {
+ super(
+ seriesPath,
+ allSensors,
+ dataType,
+ context,
+ seqFileResource,
+ unseqFileResource,
+ timeFilter,
+ valueFilter,
+ ascending);
+ this.allSensors.add(seriesPath.getMeasurement());
+ this.vectorPartialPath = (VectorPartialPath) seriesPath;
+ }
+
+ @Override
+ protected void unpackSeqTsFileResource() throws IOException {
+ TsFileResource resource = orderUtils.getNextSeqFileResource(seqFileResource, true);
+ TimeseriesMetadata timeseriesMetadata =
+ FileLoaderUtils.loadTimeSeriesMetadata(
+ resource, vectorPartialPath, context, getAnyFilter(), allSensors);
+ if (timeseriesMetadata != null) {
+ timeseriesMetadata.setSeq(true);
+ List<TimeseriesMetadata> valueTimeseriesMetadataList = new ArrayList<>();
+ for (PartialPath subSensor : vectorPartialPath.getSubSensorsPathList()) {
+ TimeseriesMetadata valueTimeSeriesMetadata =
+ FileLoaderUtils.loadTimeSeriesMetadata(
+ resource, subSensor, context, getAnyFilter(), allSensors);
+ if (valueTimeSeriesMetadata == null) {
+ throw new IOException("File doesn't contains value");
+ }
+ valueTimeSeriesMetadata.setSeq(true);
+ valueTimeseriesMetadataList.add(valueTimeSeriesMetadata);
+ }
+ VectorTimeSeriesMetadata vectorTimeSeriesMetadata =
+ new VectorTimeSeriesMetadata(timeseriesMetadata, valueTimeseriesMetadataList);
+ seqTimeSeriesMetadata.add(vectorTimeSeriesMetadata);
+ }
+ }
+
+ @Override
+ protected void unpackUnseqTsFileResource() throws IOException {
+ TsFileResource resource = unseqFileResource.remove(0);
+ TimeseriesMetadata timeseriesMetadata =
+ FileLoaderUtils.loadTimeSeriesMetadata(
+ resource, vectorPartialPath, context, getAnyFilter(), allSensors);
+ if (timeseriesMetadata != null) {
+ timeseriesMetadata.setModified(true);
+ timeseriesMetadata.setSeq(false);
+ List<TimeseriesMetadata> valueTimeseriesMetadataList = new ArrayList<>();
+ for (PartialPath subSensor : vectorPartialPath.getSubSensorsPathList()) {
+ TimeseriesMetadata valueTimeSeriesMetadata =
+ FileLoaderUtils.loadTimeSeriesMetadata(
+ resource, subSensor, context, getAnyFilter(), allSensors);
+ if (valueTimeSeriesMetadata == null) {
+ throw new IOException("File contains value");
+ }
+ timeseriesMetadata.setModified(true);
+ valueTimeSeriesMetadata.setSeq(false);
+ valueTimeseriesMetadataList.add(valueTimeSeriesMetadata);
+ }
+ VectorTimeSeriesMetadata vectorTimeSeriesMetadata =
+ new VectorTimeSeriesMetadata(timeseriesMetadata, valueTimeseriesMetadataList);
+ unSeqTimeSeriesMetadata.add(vectorTimeSeriesMetadata);
+ }
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/timegenerator/ServerTimeGenerator.java b/server/src/main/java/org/apache/iotdb/db/query/timegenerator/ServerTimeGenerator.java
index 5a9ce98..403b0e9 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/timegenerator/ServerTimeGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/timegenerator/ServerTimeGenerator.java
@@ -22,6 +22,7 @@
import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor;
import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
import org.apache.iotdb.db.metadata.PartialPath;
import org.apache.iotdb.db.qp.physical.crud.RawDataQueryPlan;
import org.apache.iotdb.db.query.context.QueryContext;
@@ -93,6 +94,12 @@
@Override
protected IBatchReader generateNewBatchReader(SingleSeriesExpression expression)
throws IOException {
+ try {
+ expression.setSeriesPath(
+ IoTDB.metaManager.transformPath((PartialPath) expression.getSeriesPath()));
+ } catch (MetadataException e) {
+ throw new IOException(e);
+ }
Filter valueFilter = expression.getFilter();
PartialPath path = (PartialPath) expression.getSeriesPath();
TSDataType dataType;
diff --git a/server/src/main/java/org/apache/iotdb/db/rescon/PrimitiveArrayManager.java b/server/src/main/java/org/apache/iotdb/db/rescon/PrimitiveArrayManager.java
index aa6c264..c0acf62 100644
--- a/server/src/main/java/org/apache/iotdb/db/rescon/PrimitiveArrayManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/rescon/PrimitiveArrayManager.java
@@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.rescon;
import org.apache.iotdb.db.conf.IoTDBConfig;
@@ -36,7 +37,7 @@
/** Manage all primitive data list in memory, including get and release operation. */
public class PrimitiveArrayManager {
- /** data type -> ArrayDeque<Array> */
+ /** data type -> ArrayDeque of primitive arrays. */
private static final Map<TSDataType, ArrayDeque<Object>> bufferedArraysMap =
new EnumMap<>(TSDataType.class);
@@ -71,6 +72,7 @@
bufferedArraysMap.put(TSDataType.FLOAT, new ArrayDeque<>());
bufferedArraysMap.put(TSDataType.DOUBLE, new ArrayDeque<>());
bufferedArraysMap.put(TSDataType.TEXT, new ArrayDeque<>());
+ bufferedArraysMap.put(TSDataType.VECTOR, new ArrayDeque<>());
}
private PrimitiveArrayManager() {
@@ -127,6 +129,9 @@
case TEXT:
dataArray = new Binary[ARRAY_SIZE];
break;
+ case VECTOR:
+ dataArray = new byte[ARRAY_SIZE][];
+ break;
default:
throw new UnSupportedDataTypeException(dataType.toString());
}
diff --git a/server/src/main/java/org/apache/iotdb/db/rescon/SystemInfo.java b/server/src/main/java/org/apache/iotdb/db/rescon/SystemInfo.java
index 9e20d51..25ad6be 100644
--- a/server/src/main/java/org/apache/iotdb/db/rescon/SystemInfo.java
+++ b/server/src/main/java/org/apache/iotdb/db/rescon/SystemInfo.java
@@ -247,4 +247,20 @@
FLUSH_THERSHOLD = memorySizeForWrite * config.getFlushProportion();
REJECT_THERSHOLD = memorySizeForWrite * config.getRejectProportion();
}
+
+ public long getTotalMemTableSize() {
+ return totalSgMemCost;
+ }
+
+ public double getFlushThershold() {
+ return FLUSH_THERSHOLD;
+ }
+
+ public double getRejectThershold() {
+ return REJECT_THERSHOLD;
+ }
+
+ public int flushingMemTableNum() {
+ return FlushManager.getInstance().getNumberOfWorkingTasks();
+ }
}
diff --git a/server/src/main/java/org/apache/iotdb/db/rescon/TVListAllocator.java b/server/src/main/java/org/apache/iotdb/db/rescon/TVListAllocator.java
index 492de33..b90c33e 100644
--- a/server/src/main/java/org/apache/iotdb/db/rescon/TVListAllocator.java
+++ b/server/src/main/java/org/apache/iotdb/db/rescon/TVListAllocator.java
@@ -29,6 +29,7 @@
import java.util.ArrayDeque;
import java.util.EnumMap;
+import java.util.List;
import java.util.Map;
import java.util.Queue;
@@ -51,14 +52,24 @@
return list != null ? list : TVList.newList(dataType);
}
- public synchronized void release(TSDataType dataType, TVList list) {
- list.clear();
- tvListCache.get(dataType).add(list);
+ public synchronized TVList allocate(List<TSDataType> dataTypes) {
+ return TVList.newVectorList(dataTypes);
}
+ /** For non-vector types. */
+ public synchronized void release(TSDataType dataType, TVList list) {
+ list.clear();
+ if (dataType != TSDataType.VECTOR) {
+ tvListCache.get(list.getDataType()).add(list);
+ }
+ }
+
+ /** For VECTOR type only. */
public synchronized void release(TVList list) {
list.clear();
- tvListCache.get(list.getDataType()).add(list);
+ if (list.getDataType() != TSDataType.VECTOR) {
+ tvListCache.get(list.getDataType()).add(list);
+ }
}
@Override
diff --git a/server/src/main/java/org/apache/iotdb/db/service/ServiceType.java b/server/src/main/java/org/apache/iotdb/db/service/ServiceType.java
index 9747cf4..48baf39 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/ServiceType.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/ServiceType.java
@@ -51,7 +51,8 @@
FLUSH_SERVICE(
"Flush ServerService", generateJmxName("org.apache.iotdb.db.engine.pool", "Flush Manager")),
- CLUSTER_MONITOR_SERVICE("Cluster Monitor ServerService", "Cluster Monitor");
+ CLUSTER_MONITOR_SERVICE("Cluster Monitor ServerService", "Cluster Monitor"),
+ SYSTEMINFO_SERVICE("MemTable Monitor Service", "MemTable, Monitor");
private final String name;
private final String jmxName;
diff --git a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
index 2821aa2..d54746e 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
@@ -50,6 +50,7 @@
import org.apache.iotdb.db.qp.physical.crud.AggregationPlan;
import org.apache.iotdb.db.qp.physical.crud.AlignByDevicePlan;
import org.apache.iotdb.db.qp.physical.crud.AlignByDevicePlan.MeasurementType;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
import org.apache.iotdb.db.qp.physical.crud.GroupByTimePlan;
import org.apache.iotdb.db.qp.physical.crud.InsertMultiTabletPlan;
@@ -60,9 +61,11 @@
import org.apache.iotdb.db.qp.physical.crud.LastQueryPlan;
import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
import org.apache.iotdb.db.qp.physical.crud.RawDataQueryPlan;
+import org.apache.iotdb.db.qp.physical.crud.SetDeviceTemplatePlan;
import org.apache.iotdb.db.qp.physical.crud.UDFPlan;
import org.apache.iotdb.db.qp.physical.crud.UDTFPlan;
import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateMultiTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.DeleteStorageGroupPlan;
@@ -91,6 +94,8 @@
import org.apache.iotdb.service.rpc.thrift.TSCancelOperationReq;
import org.apache.iotdb.service.rpc.thrift.TSCloseOperationReq;
import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateAlignedTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateDeviceTemplateReq;
import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
@@ -116,6 +121,7 @@
import org.apache.iotdb.service.rpc.thrift.TSQueryDataSet;
import org.apache.iotdb.service.rpc.thrift.TSQueryNonAlignDataSet;
import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetDeviceTemplateReq;
import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
import org.apache.iotdb.service.rpc.thrift.TSStatus;
import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
@@ -1629,7 +1635,9 @@
insertTabletPlan.setTimes(QueryDataSetUtils.readTimesFromBuffer(req.timestamps, req.size));
insertTabletPlan.setColumns(
QueryDataSetUtils.readValuesFromBuffer(
- req.values, req.types, req.measurements.size(), req.size));
+ 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);
@@ -1675,6 +1683,9 @@
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));
return insertTabletPlan;
@@ -1766,6 +1777,58 @@
}
}
+ @Override
+ public TSStatus createAlignedTimeseries(TSCreateAlignedTimeseriesReq req) {
+ try {
+ if (!checkLogin(req.getSessionId())) {
+ return RpcUtils.getStatus(TSStatusCode.NOT_LOGIN_ERROR);
+ }
+
+ // if measurements.size() == 1, convert to create timeseries
+ if (req.measurements.size() == 1) {
+ return createTimeseries(
+ new TSCreateTimeseriesReq(
+ req.sessionId,
+ req.devicePath + "." + req.measurements.get(0),
+ req.dataTypes.get(0),
+ req.encodings.get(0),
+ req.compressor));
+ }
+
+ if (AUDIT_LOGGER.isDebugEnabled()) {
+ AUDIT_LOGGER.debug(
+ "Session-{} create aligned timeseries {}.{}",
+ currSessionId.get(),
+ req.getDevicePath(),
+ req.getMeasurements());
+ }
+
+ List<TSDataType> dataTypes = new ArrayList<>();
+ for (int dataType : req.dataTypes) {
+ dataTypes.add(TSDataType.values()[dataType]);
+ }
+ List<TSEncoding> encodings = new ArrayList<>();
+ for (int encoding : req.encodings) {
+ encodings.add(TSEncoding.values()[encoding]);
+ }
+
+ CreateAlignedTimeSeriesPlan plan =
+ new CreateAlignedTimeSeriesPlan(
+ new PartialPath(req.devicePath),
+ req.measurements,
+ dataTypes,
+ encodings,
+ CompressionType.values()[req.compressor],
+ req.measurementAlias);
+
+ TSStatus status = checkAuthority(plan, req.getSessionId());
+ return status != null ? status : executeNonQueryPlan(plan);
+ } catch (Exception e) {
+ return onNPEOrUnexpectedException(
+ e, "creating aligned timeseries", TSStatusCode.EXECUTE_STATEMENT_ERROR);
+ }
+ }
+
@SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
@Override
public TSStatus createMultiTimeseries(TSCreateMultiTimeseriesReq req) {
@@ -1784,8 +1847,8 @@
CreateMultiTimeSeriesPlan multiPlan = new CreateMultiTimeSeriesPlan();
List<PartialPath> paths = new ArrayList<>(req.paths.size());
- List<TSDataType> dataTypes = new ArrayList<>(req.paths.size());
- List<TSEncoding> encodings = new ArrayList<>(req.paths.size());
+ List<TSDataType> dataTypes = new ArrayList<>(req.dataTypes.size());
+ List<TSEncoding> encodings = new ArrayList<>(req.dataTypes.size());
List<CompressionType> compressors = new ArrayList<>(req.paths.size());
List<String> alias = null;
if (req.measurementAliasList != null) {
@@ -1816,8 +1879,6 @@
}
paths.add(new PartialPath(req.paths.get(i)));
- dataTypes.add(TSDataType.values()[req.dataTypes.get(i)]);
- encodings.add(TSEncoding.values()[req.encodings.get(i)]);
compressors.add(CompressionType.values()[req.compressors.get(i)]);
if (alias != null) {
alias.add(req.measurementAliasList.get(i));
@@ -1832,6 +1893,10 @@
attributes.add(req.attributesList.get(i));
}
}
+ for (int i = 0; i < req.dataTypes.size(); i++) {
+ dataTypes.add(TSDataType.values()[req.dataTypes.get(i)]);
+ encodings.add(TSEncoding.values()[req.encodings.get(i)]);
+ }
multiPlan.setPaths(paths);
multiPlan.setDataTypes(dataTypes);
@@ -1880,6 +1945,79 @@
return statementId;
}
+ @Override
+ public TSStatus createDeviceTemplate(TSCreateDeviceTemplateReq req) throws TException {
+ try {
+ if (!checkLogin(req.getSessionId())) {
+ return RpcUtils.getStatus(TSStatusCode.NOT_LOGIN_ERROR);
+ }
+
+ if (AUDIT_LOGGER.isDebugEnabled()) {
+ AUDIT_LOGGER.debug(
+ "Session-{} create device template {}.{}.{}.{}.{}",
+ currSessionId.get(),
+ req.getName(),
+ req.getMeasurements(),
+ req.getDataTypes(),
+ req.getEncodings(),
+ req.getCompressors());
+ }
+
+ List<List<TSDataType>> dataTypes = new ArrayList<>();
+ for (List<Integer> list : req.getDataTypes()) {
+ List<TSDataType> dataTypesList = new ArrayList<>();
+ for (int dataType : list) {
+ dataTypesList.add(TSDataType.values()[dataType]);
+ }
+ dataTypes.add(dataTypesList);
+ }
+
+ List<List<TSEncoding>> encodings = new ArrayList<>();
+ for (List<Integer> list : req.getEncodings()) {
+ List<TSEncoding> encodingsList = new ArrayList<>();
+ for (int encoding : list) {
+ encodingsList.add(TSEncoding.values()[encoding]);
+ }
+ encodings.add(encodingsList);
+ }
+
+ List<CompressionType> compressionTypes = new ArrayList<>();
+ for (int compressType : req.getCompressors()) {
+ compressionTypes.add(CompressionType.values()[compressType]);
+ }
+
+ CreateTemplatePlan plan =
+ new CreateTemplatePlan(
+ req.getName(), req.getMeasurements(), dataTypes, encodings, compressionTypes);
+
+ TSStatus status = checkAuthority(plan, req.getSessionId());
+ return status != null ? status : executeNonQueryPlan(plan);
+ } catch (Exception e) {
+ return onNPEOrUnexpectedException(
+ e, "creating aligned timeseries", TSStatusCode.EXECUTE_STATEMENT_ERROR);
+ }
+ }
+
+ @Override
+ public TSStatus setDeviceTemplate(TSSetDeviceTemplateReq req) throws TException {
+ if (!checkLogin(req.getSessionId())) {
+ return RpcUtils.getStatus(TSStatusCode.NOT_LOGIN_ERROR);
+ }
+
+ if (AUDIT_LOGGER.isDebugEnabled()) {
+ AUDIT_LOGGER.debug(
+ "Session-{} set device template {}.{}",
+ currSessionId.get(),
+ req.getTemplateName(),
+ req.getPrefixPath());
+ }
+
+ SetDeviceTemplatePlan plan = new SetDeviceTemplatePlan(req.templateName, req.prefixPath);
+
+ TSStatus status = checkAuthority(plan, req.getSessionId());
+ return status != null ? status : executeNonQueryPlan(plan);
+ }
+
private TSStatus checkAuthority(PhysicalPlan plan, long sessionId) {
List<PartialPath> paths = plan.getPaths();
try {
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/TsFileRewriteTool.java b/server/src/main/java/org/apache/iotdb/db/tools/TsFileRewriteTool.java
index 1cfd792..f5e18c0 100644
--- a/server/src/main/java/org/apache/iotdb/db/tools/TsFileRewriteTool.java
+++ b/server/src/main/java/org/apache/iotdb/db/tools/TsFileRewriteTool.java
@@ -43,6 +43,7 @@
import org.apache.iotdb.tsfile.utils.Binary;
import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
@@ -144,7 +145,7 @@
List<List<ByteBuffer>> pageDataInChunkGroup = new ArrayList<>();
List<List<Boolean>> needToDecodeInfoInChunkGroup = new ArrayList<>();
byte marker;
- List<MeasurementSchema> measurementSchemaList = new ArrayList<>();
+ List<IMeasurementSchema> measurementSchemaList = new ArrayList<>();
String lastChunkGroupDeviceId = null;
try {
while ((marker = reader.readMarker()) != MetaMarker.SEPARATOR) {
@@ -290,14 +291,14 @@
*/
protected void rewrite(
String deviceId,
- List<MeasurementSchema> schemas,
+ List<IMeasurementSchema> schemas,
List<List<PageHeader>> pageHeadersInChunkGroup,
List<List<ByteBuffer>> dataInChunkGroup,
List<List<Boolean>> needToDecodeInfoInChunkGroup)
throws IOException, PageException {
- Map<Long, Map<MeasurementSchema, ChunkWriterImpl>> chunkWritersInChunkGroup = new HashMap<>();
+ Map<Long, Map<IMeasurementSchema, ChunkWriterImpl>> chunkWritersInChunkGroup = new HashMap<>();
for (int i = 0; i < schemas.size(); i++) {
- MeasurementSchema schema = schemas.get(i);
+ IMeasurementSchema schema = schemas.get(i);
List<ByteBuffer> pageDataInChunk = dataInChunkGroup.get(i);
List<PageHeader> pageHeadersInChunk = pageHeadersInChunkGroup.get(i);
List<Boolean> needToDecodeInfoInChunk = needToDecodeInfoInChunkGroup.get(i);
@@ -317,7 +318,7 @@
}
}
- for (Entry<Long, Map<MeasurementSchema, ChunkWriterImpl>> entry :
+ for (Entry<Long, Map<IMeasurementSchema, ChunkWriterImpl>> entry :
chunkWritersInChunkGroup.entrySet()) {
long partitionId = entry.getKey();
TsFileIOWriter tsFileIOWriter = partitionWriterMap.get(partitionId);
@@ -369,15 +370,15 @@
}
protected void writePageInToFile(
- MeasurementSchema schema,
+ IMeasurementSchema schema,
PageHeader pageHeader,
ByteBuffer pageData,
- Map<Long, Map<MeasurementSchema, ChunkWriterImpl>> chunkWritersInChunkGroup,
+ Map<Long, Map<IMeasurementSchema, ChunkWriterImpl>> chunkWritersInChunkGroup,
boolean isOnlyOnePageChunk)
throws PageException {
long partitionId = StorageEngine.getTimePartition(pageHeader.getStartTime());
getOrDefaultTsFileIOWriter(oldTsFile, partitionId);
- Map<MeasurementSchema, ChunkWriterImpl> chunkWriters =
+ Map<IMeasurementSchema, ChunkWriterImpl> chunkWriters =
chunkWritersInChunkGroup.getOrDefault(partitionId, new HashMap<>());
ChunkWriterImpl chunkWriter = chunkWriters.getOrDefault(schema, new ChunkWriterImpl(schema));
chunkWriter.writePageHeaderAndDataIntoBuff(pageData, pageHeader, isOnlyOnePageChunk);
@@ -386,9 +387,9 @@
}
protected void decodeAndWritePageInToFiles(
- MeasurementSchema schema,
+ IMeasurementSchema schema,
ByteBuffer pageData,
- Map<Long, Map<MeasurementSchema, ChunkWriterImpl>> chunkWritersInChunkGroup)
+ Map<Long, Map<IMeasurementSchema, ChunkWriterImpl>> chunkWritersInChunkGroup)
throws IOException {
valueDecoder.reset();
PageReader pageReader =
@@ -399,35 +400,35 @@
protected void rewritePageIntoFiles(
BatchData batchData,
- MeasurementSchema schema,
- Map<Long, Map<MeasurementSchema, ChunkWriterImpl>> chunkWritersInChunkGroup) {
+ IMeasurementSchema schema,
+ Map<Long, Map<IMeasurementSchema, ChunkWriterImpl>> chunkWritersInChunkGroup) {
while (batchData.hasCurrent()) {
long time = batchData.currentTime();
Object value = batchData.currentValue();
long partitionId = StorageEngine.getTimePartition(time);
- Map<MeasurementSchema, ChunkWriterImpl> chunkWriters =
+ Map<IMeasurementSchema, ChunkWriterImpl> chunkWriters =
chunkWritersInChunkGroup.getOrDefault(partitionId, new HashMap<>());
ChunkWriterImpl chunkWriter = chunkWriters.getOrDefault(schema, new ChunkWriterImpl(schema));
getOrDefaultTsFileIOWriter(oldTsFile, partitionId);
switch (schema.getType()) {
case INT32:
- chunkWriter.write(time, (int) value);
+ chunkWriter.write(time, (int) value, false);
break;
case INT64:
- chunkWriter.write(time, (long) value);
+ chunkWriter.write(time, (long) value, false);
break;
case FLOAT:
- chunkWriter.write(time, (float) value);
+ chunkWriter.write(time, (float) value, false);
break;
case DOUBLE:
- chunkWriter.write(time, (double) value);
+ chunkWriter.write(time, (double) value, false);
break;
case BOOLEAN:
- chunkWriter.write(time, (boolean) value);
+ chunkWriter.write(time, (boolean) value, false);
break;
case TEXT:
- chunkWriter.write(time, (Binary) value);
+ chunkWriter.write(time, (Binary) value, false);
break;
default:
throw new UnSupportedDataTypeException(
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/mlog/MLogParser.java b/server/src/main/java/org/apache/iotdb/db/tools/mlog/MLogParser.java
index 85af420..6b570ee 100644
--- a/server/src/main/java/org/apache/iotdb/db/tools/mlog/MLogParser.java
+++ b/server/src/main/java/org/apache/iotdb/db/tools/mlog/MLogParser.java
@@ -24,6 +24,7 @@
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
import org.apache.iotdb.db.qp.physical.sys.ChangeAliasPlan;
import org.apache.iotdb.db.qp.physical.sys.ChangeTagOffsetPlan;
+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.MNodePlan;
import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
@@ -161,6 +162,9 @@
mLogTxtWriter.createTimeseries(
(CreateTimeSeriesPlan) plan, ((CreateTimeSeriesPlan) plan).getTagOffset());
break;
+ case CREATE_ALIGNED_TIMESERIES:
+ mLogTxtWriter.createAlignedTimeseries((CreateAlignedTimeSeriesPlan) plan);
+ break;
case DELETE_TIMESERIES:
for (PartialPath partialPath : plan.getPaths()) {
mLogTxtWriter.deleteTimeseries(partialPath.getFullPath());
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java b/server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
index 8549858..5751218 100644
--- a/server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
+++ b/server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
@@ -35,6 +35,7 @@
import org.apache.iotdb.tsfile.v2.read.TsFileSequenceReaderForV2;
import org.apache.iotdb.tsfile.v2.read.reader.page.PageReaderV2;
import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
@@ -79,11 +80,7 @@
}
}
- /**
- * upgrade file resource
- *
- * @throws IOException, WriteProcessException
- */
+ /** upgrade file resource */
@SuppressWarnings({"squid:S3776", "deprecation"}) // Suppress high Cognitive Complexity warning
private void upgradeFile(List<TsFileResource> upgradedResources)
throws IOException, WriteProcessException {
@@ -101,13 +98,13 @@
List<List<ByteBuffer>> pageDataInChunkGroup = new ArrayList<>();
List<List<Boolean>> needToDecodeInfoInChunkGroup = new ArrayList<>();
byte marker;
- List<MeasurementSchema> measurementSchemaList = new ArrayList<>();
+ List<IMeasurementSchema> measurementSchemaList = new ArrayList<>();
try {
while ((marker = reader.readMarker()) != MetaMarker.SEPARATOR) {
switch (marker) {
case MetaMarker.CHUNK_HEADER:
ChunkHeader header = ((TsFileSequenceReaderForV2) reader).readChunkHeader();
- MeasurementSchema measurementSchema =
+ IMeasurementSchema measurementSchema =
new MeasurementSchema(
header.getMeasurementID(),
header.getDataType(),
@@ -239,9 +236,9 @@
@Override
protected void decodeAndWritePageInToFiles(
- MeasurementSchema schema,
+ IMeasurementSchema schema,
ByteBuffer pageData,
- Map<Long, Map<MeasurementSchema, ChunkWriterImpl>> chunkWritersInChunkGroup)
+ Map<Long, Map<IMeasurementSchema, ChunkWriterImpl>> chunkWritersInChunkGroup)
throws IOException {
valueDecoder.reset();
PageReaderV2 pageReader =
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/CommonUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/CommonUtils.java
index bb82e71..ac98b54 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/CommonUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/CommonUtils.java
@@ -92,6 +92,9 @@
public static Object parseValue(TSDataType dataType, String value) throws QueryProcessException {
try {
+ if ("null".equals(value) || "NULL".equals(value)) {
+ return null;
+ }
switch (dataType) {
case BOOLEAN:
return parseBoolean(value);
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java
index 0a55b54..6464c4a 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java
@@ -29,8 +29,10 @@
import org.apache.iotdb.db.query.reader.chunk.metadata.DiskChunkMetadataLoader;
import org.apache.iotdb.db.query.reader.chunk.metadata.MemChunkMetadataLoader;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ITimeSeriesMetadata;
import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
-import org.apache.iotdb.tsfile.file.metadata.TsFileMetadata;
+import org.apache.iotdb.tsfile.file.metadata.VectorChunkMetadata;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import org.apache.iotdb.tsfile.read.common.Chunk;
import org.apache.iotdb.tsfile.read.common.Path;
@@ -39,8 +41,10 @@
import org.apache.iotdb.tsfile.read.reader.IChunkReader;
import org.apache.iotdb.tsfile.read.reader.IPageReader;
import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.read.reader.chunk.VectorChunkReader;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.List;
import java.util.Map.Entry;
import java.util.Set;
@@ -141,7 +145,7 @@
*
* @param timeSeriesMetadata the corresponding TimeSeriesMetadata in that file.
*/
- public static List<ChunkMetadata> loadChunkMetadataList(TimeseriesMetadata timeSeriesMetadata)
+ public static List<IChunkMetadata> loadChunkMetadataList(ITimeSeriesMetadata timeSeriesMetadata)
throws IOException {
return timeSeriesMetadata.loadChunkMetadataList();
}
@@ -152,8 +156,8 @@
* @param chunkMetaData the corresponding chunk metadata
* @param timeFilter it should be a TimeFilter instead of a ValueFilter
*/
- public static List<IPageReader> loadPageReaderList(ChunkMetadata chunkMetaData, Filter timeFilter)
- throws IOException {
+ public static List<IPageReader> loadPageReaderList(
+ IChunkMetadata chunkMetaData, Filter timeFilter) throws IOException {
if (chunkMetaData == null) {
throw new IOException("Can't init null chunkMeta");
}
@@ -163,22 +167,24 @@
MemChunkLoader memChunkLoader = (MemChunkLoader) chunkLoader;
chunkReader = new MemChunkReader(memChunkLoader.getChunk(), timeFilter);
} else {
- Chunk chunk = chunkLoader.loadChunk(chunkMetaData);
- chunk.setFromOldFile(chunkMetaData.isFromOldTsFile());
- chunkReader = new ChunkReader(chunk, timeFilter);
- chunkReader.hasNextSatisfiedPage();
+ if (chunkMetaData instanceof ChunkMetadata) {
+ Chunk chunk = chunkLoader.loadChunk((ChunkMetadata) chunkMetaData);
+ chunk.setFromOldFile(chunkMetaData.isFromOldTsFile());
+ chunkReader = new ChunkReader(chunk, timeFilter);
+ chunkReader.hasNextSatisfiedPage();
+ } else {
+ VectorChunkMetadata vectorChunkMetadata = (VectorChunkMetadata) chunkMetaData;
+ Chunk timeChunk = vectorChunkMetadata.getTimeChunk();
+ List<Chunk> valueChunkList = vectorChunkMetadata.getValueChunkList();
+ chunkReader = new VectorChunkReader(timeChunk, valueChunkList, timeFilter);
+ }
}
return chunkReader.loadPageReaderList();
}
- public static List<ChunkMetadata> getChunkMetadataList(Path path, String filePath)
+ public static List<IChunkMetadata> getChunkMetadataList(Path path, String filePath)
throws IOException {
TsFileSequenceReader tsFileReader = FileReaderManager.getInstance().get(filePath, true);
- return tsFileReader.getChunkMetadataList(path);
- }
-
- public static TsFileMetadata getTsFileMetadata(String filePath) throws IOException {
- TsFileSequenceReader reader = FileReaderManager.getInstance().get(filePath, true);
- return reader.readFileMetadata();
+ return new ArrayList<>(tsFileReader.getChunkMetadataList(path));
}
}
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 2d8488b..667a984 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
@@ -31,10 +31,13 @@
import org.apache.iotdb.tsfile.write.record.datapoint.IntDataPoint;
import org.apache.iotdb.tsfile.write.record.datapoint.LongDataPoint;
import org.apache.iotdb.tsfile.write.record.datapoint.StringDataPoint;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.util.List;
+
// Notice : methods in this class may not be accurate.
public class MemUtils {
@@ -47,22 +50,28 @@
* the size will be added to memtable before inserting.
*/
public static long getRecordSize(TSDataType dataType, Object value, boolean addingTextDataSize) {
- switch (dataType) {
- case INT32:
- return 8L + 4L;
- case INT64:
- return 8L + 8L;
- case FLOAT:
- return 8L + 4L;
- case DOUBLE:
- return 8L + 8L;
- case BOOLEAN:
- return 8L + 1L;
- case TEXT:
- return 8L + (addingTextDataSize ? getBinarySize((Binary) value) : 0);
- default:
- return 8L + 8L;
+ if (dataType == TSDataType.TEXT) {
+ return 8L + (addingTextDataSize ? getBinarySize((Binary) value) : 0);
}
+ return 8L + dataType.getDataTypeSize();
+ }
+
+ /**
+ * function for getting the vector value size. If mem control enabled, do not add text data size
+ * here, the size will be added to memtable before inserting.
+ */
+ public static long getVectorRecordSize(
+ List<TSDataType> dataTypes, Object[] value, boolean addingTextDataSize) {
+ // time and index size
+ long memSize = 8L + 4L;
+ for (int i = 0; i < dataTypes.size(); i++) {
+ if (dataTypes.get(i) == TSDataType.TEXT) {
+ memSize += (addingTextDataSize ? getBinarySize((Binary) value[i]) : 0);
+ } else {
+ memSize += dataTypes.get(i).getDataTypeSize();
+ }
+ }
+ return memSize;
}
public static long getBinarySize(Binary value) {
@@ -82,42 +91,72 @@
* If mem control enabled, do not add text data size here, the size will be added to memtable
* before inserting.
*/
+ @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
public static long getRecordSize(
InsertTabletPlan insertTabletPlan, int start, int end, boolean addingTextDataSize) {
+ if (insertTabletPlan.getMeasurementMNodes() == null) {
+ return getRecordSizeForTest(insertTabletPlan, start, end, addingTextDataSize);
+ }
+ if (start >= end) {
+ return 0L;
+ }
+ long memSize = 0;
+ int columnCount = 0;
+ for (int i = 0; i < insertTabletPlan.getMeasurementMNodes().length; i++) {
+ if (insertTabletPlan.getMeasurementMNodes()[i] == null) {
+ columnCount++;
+ continue;
+ }
+ IMeasurementSchema schema = insertTabletPlan.getMeasurementMNodes()[i].getSchema();
+ if (schema.getType() == TSDataType.VECTOR) {
+ // time and index column memSize
+ memSize += (end - start) * (8L + 4L);
+ // value columns memSize
+ for (TSDataType type : schema.getValueTSDataTypeList()) {
+ if (type == TSDataType.TEXT && addingTextDataSize) {
+ for (int j = start; j < end; j++) {
+ memSize += getBinarySize(((Binary[]) insertTabletPlan.getColumns()[columnCount])[j]);
+ }
+ } else {
+ memSize += (end - start) * type.getDataTypeSize();
+ }
+ columnCount++;
+ }
+ } else {
+ // time column memSize
+ memSize += (end - start) * 8L;
+ if (insertTabletPlan.getDataTypes()[columnCount] == TSDataType.TEXT && addingTextDataSize) {
+ for (int j = start; j < end; j++) {
+ memSize += getBinarySize(((Binary[]) insertTabletPlan.getColumns()[columnCount])[j]);
+ }
+ } else {
+ memSize += (end - start) * insertTabletPlan.getDataTypes()[columnCount].getDataTypeSize();
+ }
+ columnCount++;
+ }
+ }
+ return memSize;
+ }
+
+ /**
+ * This method is for test only. This reason is the InsertTabletPlan in tests may doesn't have
+ * MeasurementMNodes
+ */
+ public static long getRecordSizeForTest(
+ 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.getDataTypes()[i] == null) {
- continue;
- }
- switch (insertTabletPlan.getDataTypes()[i]) {
- case INT32:
- memSize += (end - start) * (8L + 4L);
- break;
- case INT64:
- memSize += (end - start) * (8L + 8L);
- break;
- case FLOAT:
- memSize += (end - start) * (8L + 4L);
- break;
- case DOUBLE:
- memSize += (end - start) * (8L + 8L);
- break;
- case BOOLEAN:
- memSize += (end - start) * (8L + 1L);
- break;
- case TEXT:
- memSize += (end - start) * 8L;
- if (addingTextDataSize) {
- for (int j = start; j < end; j++) {
- memSize += getBinarySize(((Binary[]) insertTabletPlan.getColumns()[i])[j]);
- }
- }
- break;
- default:
- memSize += (end - start) * (8L + 8L);
+ // 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;
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/MergeUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/MergeUtils.java
index 973cb24..2dbaca3 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/MergeUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/MergeUtils.java
@@ -53,22 +53,25 @@
public static void writeTVPair(TimeValuePair timeValuePair, IChunkWriter chunkWriter) {
switch (chunkWriter.getDataType()) {
case TEXT:
- chunkWriter.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getBinary());
+ chunkWriter.write(
+ timeValuePair.getTimestamp(), timeValuePair.getValue().getBinary(), false);
break;
case DOUBLE:
- chunkWriter.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getDouble());
+ chunkWriter.write(
+ timeValuePair.getTimestamp(), timeValuePair.getValue().getDouble(), false);
break;
case BOOLEAN:
- chunkWriter.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getBoolean());
+ chunkWriter.write(
+ timeValuePair.getTimestamp(), timeValuePair.getValue().getBoolean(), false);
break;
case INT64:
- chunkWriter.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getLong());
+ chunkWriter.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getLong(), false);
break;
case INT32:
- chunkWriter.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getInt());
+ chunkWriter.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getInt(), false);
break;
case FLOAT:
- chunkWriter.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getFloat());
+ chunkWriter.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getFloat(), false);
break;
default:
throw new UnsupportedOperationException("Unknown data type " + chunkWriter.getDataType());
@@ -109,22 +112,22 @@
public static void writeBatchPoint(BatchData batchData, int i, IChunkWriter chunkWriter) {
switch (chunkWriter.getDataType()) {
case TEXT:
- chunkWriter.write(batchData.getTimeByIndex(i), batchData.getBinaryByIndex(i));
+ chunkWriter.write(batchData.getTimeByIndex(i), batchData.getBinaryByIndex(i), false);
break;
case DOUBLE:
- chunkWriter.write(batchData.getTimeByIndex(i), batchData.getDoubleByIndex(i));
+ chunkWriter.write(batchData.getTimeByIndex(i), batchData.getDoubleByIndex(i), false);
break;
case BOOLEAN:
- chunkWriter.write(batchData.getTimeByIndex(i), batchData.getBooleanByIndex(i));
+ chunkWriter.write(batchData.getTimeByIndex(i), batchData.getBooleanByIndex(i), false);
break;
case INT64:
- chunkWriter.write(batchData.getTimeByIndex(i), batchData.getLongByIndex(i));
+ chunkWriter.write(batchData.getTimeByIndex(i), batchData.getLongByIndex(i), false);
break;
case INT32:
- chunkWriter.write(batchData.getTimeByIndex(i), batchData.getIntByIndex(i));
+ chunkWriter.write(batchData.getTimeByIndex(i), batchData.getIntByIndex(i), false);
break;
case FLOAT:
- chunkWriter.write(batchData.getTimeByIndex(i), batchData.getFloatByIndex(i));
+ chunkWriter.write(batchData.getTimeByIndex(i), batchData.getFloatByIndex(i), false);
break;
default:
throw new UnsupportedOperationException("Unknown data type " + chunkWriter.getDataType());
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java
index 705c0cf..4c32123 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java
@@ -26,6 +26,7 @@
import org.apache.iotdb.tsfile.read.common.RowRecord;
import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.BitMap;
import org.apache.iotdb.tsfile.utils.BytesUtils;
import java.io.ByteArrayOutputStream;
@@ -183,6 +184,24 @@
return times;
}
+ public static BitMap[] readBitMapsFromBuffer(ByteBuffer buffer, int columns, int size) {
+ if (!buffer.hasRemaining()) {
+ return null;
+ }
+ BitMap[] bitMaps = new BitMap[columns];
+ for (int i = 0; i < columns; i++) {
+ boolean hasBitMap = BytesUtils.byteToBool(buffer.get());
+ if (hasBitMap) {
+ byte[] bytes = new byte[size / Byte.SIZE + 1];
+ for (int j = 0; j < bytes.length; j++) {
+ bytes[j] = buffer.get();
+ }
+ bitMaps[i] = new BitMap(size, bytes);
+ }
+ }
+ return bitMaps;
+ }
+
public static Object[] readValuesFromBuffer(
ByteBuffer buffer, List<Integer> types, int columns, int size) {
TSDataType[] dataTypes = new TSDataType[types.size()];
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
index 5c0409f..4e07b2a 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
@@ -24,7 +24,7 @@
import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
import org.apache.iotdb.db.query.filter.TsFileFilter;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.read.common.TimeRange;
import java.util.List;
@@ -46,9 +46,9 @@
*/
@SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
public static void modifyChunkMetaData(
- List<ChunkMetadata> chunkMetaData, List<Modification> modifications) {
+ List<? extends IChunkMetadata> chunkMetaData, List<Modification> modifications) {
for (int metaIndex = 0; metaIndex < chunkMetaData.size(); metaIndex++) {
- ChunkMetadata metaData = chunkMetaData.get(metaIndex);
+ IChunkMetadata metaData = chunkMetaData.get(metaIndex);
for (Modification modification : modifications) {
// When the chunkMetadata come from an old TsFile, the method modification.getFileOffset()
// is gerVersionNum actually. In this case, we compare the versions of modification and
@@ -86,7 +86,7 @@
});
}
- private static void doModifyChunkMetaData(Modification modification, ChunkMetadata metaData) {
+ private static void doModifyChunkMetaData(Modification modification, IChunkMetadata metaData) {
if (modification instanceof Deletion) {
Deletion deletion = (Deletion) modification;
metaData.insertIntoSortedDeletions(deletion.getStartTime(), deletion.getEndTime());
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java
index f040c6b..c02ac81 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java
@@ -30,6 +30,7 @@
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.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
@@ -114,7 +115,7 @@
TSDataType dataType = schema.getType();
TSEncoding encoding = schema.getEncodingType();
CompressionType compressionType = schema.getCompressor();
- MeasurementSchema measurementSchema =
+ IMeasurementSchema measurementSchema =
new MeasurementSchema(path.getMeasurement(), dataType, encoding, compressionType);
MeasurementMNode measurementMNode =
@@ -157,7 +158,6 @@
* @param measurementDataType
* @param aggregation
* @return
- * @throws MetadataException
*/
public static List<TSDataType> getAggregatedDataTypes(
List<TSDataType> measurementDataType, String aggregation) {
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/SerializeUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/SerializeUtils.java
index cb881e0..86e6c6e 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/SerializeUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/SerializeUtils.java
@@ -117,6 +117,7 @@
}
}
+ @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
public static void serializeBatchData(BatchData batchData, DataOutputStream outputStream) {
try {
int length = batchData.length();
@@ -162,12 +163,49 @@
outputStream.writeInt(batchData.getIntByIndex(i));
}
break;
+ case VECTOR:
+ for (int i = 0; i < length; i++) {
+ outputStream.writeLong(batchData.getTimeByIndex(i));
+ TsPrimitiveType[] values = batchData.getVectorByIndex(i);
+ outputStream.writeInt(values.length);
+ for (TsPrimitiveType value : values) {
+ if (value == null) {
+ outputStream.write(0);
+ } else {
+ outputStream.write(1);
+ outputStream.write(value.getDataType().serialize());
+ switch (value.getDataType()) {
+ case BOOLEAN:
+ outputStream.writeBoolean(value.getBoolean());
+ break;
+ case DOUBLE:
+ outputStream.writeDouble(value.getDouble());
+ break;
+ case FLOAT:
+ outputStream.writeFloat(value.getFloat());
+ break;
+ case TEXT:
+ Binary binary = value.getBinary();
+ outputStream.writeInt(binary.getLength());
+ outputStream.write(binary.getValues());
+ break;
+ case INT64:
+ outputStream.writeLong(value.getLong());
+ break;
+ case INT32:
+ outputStream.writeInt(value.getInt());
+ break;
+ }
+ }
+ }
+ }
}
} catch (IOException ignored) {
// ignored
}
}
+ @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
public static BatchData deserializeBatchData(ByteBuffer buffer) {
if (buffer == null || (buffer.limit() - buffer.position()) == 0) {
return null;
@@ -211,6 +249,42 @@
batchData.putBoolean(buffer.getLong(), buffer.get() == 1);
}
break;
+ case VECTOR:
+ for (int i = 0; i < length; i++) {
+ long time = buffer.getLong();
+ int valuesLength = buffer.getInt();
+ TsPrimitiveType[] values = new TsPrimitiveType[valuesLength];
+ for (int j = 0; j < valuesLength; j++) {
+ boolean notNull = (buffer.get() == 1);
+ if (notNull) {
+ switch (TSDataType.values()[buffer.get()]) {
+ case BOOLEAN:
+ values[j] = new TsPrimitiveType.TsBoolean(buffer.get() == 1);
+ break;
+ case DOUBLE:
+ values[j] = new TsPrimitiveType.TsDouble(buffer.getDouble());
+ break;
+ case FLOAT:
+ values[j] = new TsPrimitiveType.TsFloat(buffer.getFloat());
+ break;
+ case TEXT:
+ int len = buffer.getInt();
+ byte[] bytes = new byte[len];
+ buffer.get(bytes);
+ values[j] = new TsPrimitiveType.TsBinary(new Binary(bytes));
+ break;
+ case INT64:
+ values[j] = new TsPrimitiveType.TsLong(buffer.getLong());
+ break;
+ case INT32:
+ values[j] = new TsPrimitiveType.TsInt(buffer.getInt());
+ break;
+ }
+ }
+ }
+ batchData.putVector(time, values);
+ }
+ break;
}
return batchData;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/TypeInferenceUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/TypeInferenceUtils.java
index e4d5338..7e0c150 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/TypeInferenceUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/TypeInferenceUtils.java
@@ -80,6 +80,8 @@
} else {
return floatingStringInferType;
}
+ } else if ("null".equals(strValue) || "NULL".equals(strValue)) {
+ return null;
// "NaN" is returned if the NaN Literal is given in Parser
} else if ("NaN".equals(strValue)) {
return nanStringInferType;
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
index d24beae..8ac571f 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
@@ -27,6 +27,7 @@
import org.apache.iotdb.tsfile.read.common.TimeRange;
import org.apache.iotdb.tsfile.read.reader.IPointReader;
import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.BitMap;
import java.io.IOException;
import java.util.ArrayList;
@@ -38,7 +39,7 @@
public abstract class TVList {
protected static final int SMALL_ARRAY_LENGTH = 32;
- private static final String ERR_DATATYPE_NOT_CONSISTENT = "DataType not consistent";
+ protected static final String ERR_DATATYPE_NOT_CONSISTENT = "DataType not consistent";
protected List<long[]> timestamps;
protected int size;
@@ -79,6 +80,10 @@
return null;
}
+ public static TVList newVectorList(List<TSDataType> datatypes) {
+ return new VectorTVList(datatypes);
+ }
+
public static long tvListArrayMemSize(TSDataType type) {
long size = 0;
// time size
@@ -88,6 +93,25 @@
return size;
}
+ /**
+ * For Vector data type.
+ *
+ * @param types the types in the vector
+ * @return VectorTvListArrayMemSize
+ */
+ public static long vectorTvListArrayMemSize(List<TSDataType> types) {
+ long size = 0;
+ // time size
+ size += (long) PrimitiveArrayManager.ARRAY_SIZE * 8L;
+ // index size
+ size += (long) PrimitiveArrayManager.ARRAY_SIZE * 4L;
+ // value size
+ for (TSDataType type : types) {
+ size += (long) PrimitiveArrayManager.ARRAY_SIZE * (long) type.getDataTypeSize();
+ }
+ return size;
+ }
+
public boolean isSorted() {
return sorted;
}
@@ -137,6 +161,10 @@
throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
}
+ public void putVector(long time, Object[] value) {
+ throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
+ }
+
public void putLongs(long[] time, long[] value, int start, int end) {
throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
}
@@ -161,6 +189,10 @@
throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
}
+ public void putVectors(long[] time, BitMap[] bitMaps, Object[] value, int start, int end) {
+ throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
+ }
+
public long getLong(int index) {
throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
}
@@ -185,6 +217,14 @@
throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
}
+ public Object getVector(int index) {
+ throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
+ }
+
+ public TVList getTvListByColumnIndex(List<Integer> columnIndexList) {
+ throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
+ }
+
public abstract void sort();
public long getMinTime() {
@@ -240,9 +280,17 @@
releaseLastTimeArray();
releaseLastValueArray();
}
+ if (getDataType() == TSDataType.VECTOR) {
+ return deletedNumber * ((VectorTVList) this).getTsDataTypes().size();
+ }
return deletedNumber;
}
+ // TODO: THIS METHOLD IS FOR DELETING ONE COLUMN OF A VECTOR
+ public int delete(long lowerBound, long upperBound, int columnIndex) {
+ throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
+ }
+
protected void cloneAs(TVList cloneList) {
for (long[] timestampArray : timestamps) {
cloneList.timestamps.add(cloneTime(timestampArray));
@@ -523,10 +571,13 @@
cur++;
continue;
}
- cachedTimeValuePair = getTimeValuePair(cur, time, floatPrecision, encoding);
- hasCachedPair = true;
+ TimeValuePair tvPair = getTimeValuePair(cur, time, floatPrecision, encoding);
cur++;
- return true;
+ if (tvPair.getValue() != null) {
+ cachedTimeValuePair = tvPair;
+ hasCachedPair = true;
+ return true;
+ }
}
return false;
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/VectorTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/VectorTVList.java
new file mode 100644
index 0000000..0a3b405
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/VectorTVList.java
@@ -0,0 +1,653 @@
+/*
+ * 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.utils.datastructure;
+
+import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
+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.BitMap;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.iotdb.db.rescon.PrimitiveArrayManager.ARRAY_SIZE;
+
+public class VectorTVList extends TVList {
+
+ private List<TSDataType> dataTypes;
+
+ private List<List<Object>> values;
+
+ private List<int[]> indices;
+
+ private List<List<BitMap>> bitMaps;
+
+ private int[][] sortedIndices;
+
+ private int pivotIndex;
+
+ VectorTVList(List<TSDataType> types) {
+ super();
+ indices = new ArrayList<>(types.size());
+ dataTypes = types;
+ values = new ArrayList<>(types.size());
+ for (int i = 0; i < types.size(); i++) {
+ values.add(new ArrayList<>());
+ }
+ }
+
+ @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+ @Override
+ public void putVector(long timestamp, Object[] value) {
+ checkExpansion();
+ int arrayIndex = size / ARRAY_SIZE;
+ int elementIndex = size % ARRAY_SIZE;
+ minTime = Math.min(minTime, timestamp);
+ timestamps.get(arrayIndex)[elementIndex] = timestamp;
+ for (int i = 0; i < values.size(); i++) {
+ Object columnValue = value[i];
+ List<Object> columnValues = values.get(i);
+ if (columnValue == null) {
+ markNullValue(i, arrayIndex, elementIndex);
+ } else if (isValueMarked(indices.get(arrayIndex)[elementIndex], i)) {
+ bitMaps.get(i).get(arrayIndex).unmark(elementIndex);
+ }
+ switch (dataTypes.get(i)) {
+ case TEXT:
+ ((Binary[]) columnValues.get(arrayIndex))[elementIndex] =
+ columnValue != null ? (Binary) columnValue : Binary.EMPTY_VALUE;
+ break;
+ case FLOAT:
+ ((float[]) columnValues.get(arrayIndex))[elementIndex] =
+ columnValue != null ? (float) columnValue : Float.MIN_VALUE;
+ break;
+ case INT32:
+ ((int[]) columnValues.get(arrayIndex))[elementIndex] =
+ columnValue != null ? (int) columnValue : Integer.MIN_VALUE;
+ break;
+ case INT64:
+ ((long[]) columnValues.get(arrayIndex))[elementIndex] =
+ columnValue != null ? (long) columnValue : Long.MIN_VALUE;
+ break;
+ case DOUBLE:
+ ((double[]) columnValues.get(arrayIndex))[elementIndex] =
+ columnValue != null ? (double) columnValue : Double.MIN_VALUE;
+ break;
+ case BOOLEAN:
+ ((boolean[]) columnValues.get(arrayIndex))[elementIndex] =
+ columnValue != null && (boolean) columnValue;
+ break;
+ default:
+ break;
+ }
+ }
+ indices.get(arrayIndex)[elementIndex] = size;
+ size++;
+ if (sorted && size > 1 && timestamp < getTime(size - 2)) {
+ sorted = false;
+ }
+ }
+
+ @Override
+ public Object getVector(int index) {
+ if (index >= size) {
+ throw new ArrayIndexOutOfBoundsException(index);
+ }
+ int arrayIndex = index / ARRAY_SIZE;
+ int elementIndex = index % ARRAY_SIZE;
+ int valueIndex = indices.get(arrayIndex)[elementIndex];
+ return getVectorByValueIndex(valueIndex);
+ }
+
+ private Object getVectorByValueIndex(int valueIndex) {
+ if (valueIndex >= size) {
+ throw new ArrayIndexOutOfBoundsException(valueIndex);
+ }
+ int arrayIndex = valueIndex / ARRAY_SIZE;
+ int elementIndex = valueIndex % ARRAY_SIZE;
+ TsPrimitiveType[] vector = new TsPrimitiveType[values.size()];
+ for (int i = 0; i < values.size(); i++) {
+ List<Object> columnValues = values.get(i);
+ if (bitMaps != null
+ && bitMaps.get(i) != null
+ && bitMaps.get(i).get(arrayIndex).isMarked(elementIndex)) {
+ continue;
+ }
+ switch (dataTypes.get(i)) {
+ case TEXT:
+ vector[i] =
+ TsPrimitiveType.getByType(
+ dataTypes.get(i), ((Binary[]) columnValues.get(arrayIndex))[elementIndex]);
+ break;
+ case FLOAT:
+ vector[i] =
+ TsPrimitiveType.getByType(
+ dataTypes.get(i), ((float[]) columnValues.get(arrayIndex))[elementIndex]);
+ break;
+ case INT32:
+ vector[i] =
+ TsPrimitiveType.getByType(
+ dataTypes.get(i), ((int[]) columnValues.get(arrayIndex))[elementIndex]);
+ break;
+ case INT64:
+ vector[i] =
+ TsPrimitiveType.getByType(
+ dataTypes.get(i), ((long[]) columnValues.get(arrayIndex))[elementIndex]);
+ break;
+ case DOUBLE:
+ vector[i] =
+ TsPrimitiveType.getByType(
+ dataTypes.get(i), ((double[]) columnValues.get(arrayIndex))[elementIndex]);
+ break;
+ case BOOLEAN:
+ vector[i] =
+ TsPrimitiveType.getByType(
+ dataTypes.get(i), ((boolean[]) columnValues.get(arrayIndex))[elementIndex]);
+ break;
+ default:
+ throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
+ }
+ }
+ return TsPrimitiveType.getByType(TSDataType.VECTOR, vector);
+ }
+
+ @Override
+ public TVList getTvListByColumnIndex(List<Integer> columns) {
+ List<TSDataType> types = new ArrayList<>();
+ List<List<Object>> values = new ArrayList<>();
+ List<List<BitMap>> bitMaps = null;
+ for (int column : columns) {
+ types.add(this.dataTypes.get(column));
+ values.add(this.values.get(column));
+ if (this.bitMaps != null && this.bitMaps.get(column) != null) {
+ if (bitMaps == null) {
+ bitMaps = new ArrayList<>(columns.size());
+ for (int i = 0; i < columns.size(); i++) {
+ bitMaps.add(null);
+ }
+ }
+ bitMaps.set(columns.indexOf(column), this.bitMaps.get(column));
+ }
+ }
+ VectorTVList vectorTvList = new VectorTVList(types);
+ vectorTvList.timestamps = this.timestamps;
+ vectorTvList.indices = this.indices;
+ vectorTvList.values = values;
+ vectorTvList.bitMaps = bitMaps;
+ vectorTvList.size = this.size;
+ return vectorTvList;
+ }
+
+ /**
+ * Get the int value at the given position in VectorTvList.
+ *
+ * @param rowIndex value index inside this column
+ * @param columnIndex index of the column
+ * @return the value at this position in VectorTvList
+ */
+ public int getIntByValueIndex(int rowIndex, int columnIndex) {
+ int arrayIndex = rowIndex / ARRAY_SIZE;
+ int elementIndex = rowIndex % ARRAY_SIZE;
+ List<Object> columnValues = values.get(columnIndex);
+ return ((int[]) columnValues.get(arrayIndex))[elementIndex];
+ }
+
+ /**
+ * Get the long value at the given position in VectorTvList.
+ *
+ * @param rowIndex value index inside this column
+ * @param columnIndex index of the column
+ * @return the value at this position in VectorTvList
+ */
+ public long getLongByValueIndex(int rowIndex, int columnIndex) {
+ int arrayIndex = rowIndex / ARRAY_SIZE;
+ int elementIndex = rowIndex % ARRAY_SIZE;
+ List<Object> columnValues = values.get(columnIndex);
+ return ((long[]) columnValues.get(arrayIndex))[elementIndex];
+ }
+
+ /**
+ * Get the float value at the given position in VectorTvList.
+ *
+ * @param rowIndex value index inside this column
+ * @param columnIndex index of the column
+ * @return the value at this position in VectorTvList
+ */
+ public float getFloatByValueIndex(int rowIndex, int columnIndex) {
+ int arrayIndex = rowIndex / ARRAY_SIZE;
+ int elementIndex = rowIndex % ARRAY_SIZE;
+ List<Object> columnValues = values.get(columnIndex);
+ return ((float[]) columnValues.get(arrayIndex))[elementIndex];
+ }
+
+ /**
+ * Get the double value at the given position in VectorTvList.
+ *
+ * @param rowIndex value index inside this column
+ * @param columnIndex index of the column
+ * @return the value at this position in VectorTvList
+ */
+ public double getDoubleByValueIndex(int rowIndex, int columnIndex) {
+ int arrayIndex = rowIndex / ARRAY_SIZE;
+ int elementIndex = rowIndex % ARRAY_SIZE;
+ List<Object> columnValues = values.get(columnIndex);
+ return ((double[]) columnValues.get(arrayIndex))[elementIndex];
+ }
+
+ /**
+ * Get the Binary value at the given position in VectorTvList.
+ *
+ * @param rowIndex value index inside this column
+ * @param columnIndex index of the column
+ * @return the value at this position in VectorTvList
+ */
+ public Binary getBinaryByValueIndex(int rowIndex, int columnIndex) {
+ int arrayIndex = rowIndex / ARRAY_SIZE;
+ int elementIndex = rowIndex % ARRAY_SIZE;
+ List<Object> columnValues = values.get(columnIndex);
+ return ((Binary[]) columnValues.get(arrayIndex))[elementIndex];
+ }
+
+ /**
+ * Get the boolean value at the given position in VectorTvList.
+ *
+ * @param rowIndex value index inside this column
+ * @param columnIndex index of the column
+ * @return the value at this position in VectorTvList
+ */
+ public boolean getBooleanByValueIndex(int rowIndex, int columnIndex) {
+ int arrayIndex = rowIndex / ARRAY_SIZE;
+ int elementIndex = rowIndex % ARRAY_SIZE;
+ List<Object> columnValues = values.get(columnIndex);
+ return ((boolean[]) columnValues.get(arrayIndex))[elementIndex];
+ }
+
+ /**
+ * Get whether value is marked at the given position in VectorTvList.
+ *
+ * @param rowIndex value index inside this column
+ * @param columnIndex index of the column
+ * @return boolean
+ */
+ public boolean isValueMarked(int rowIndex, int columnIndex) {
+ if (rowIndex >= size) {
+ return false;
+ }
+ if (bitMaps == null || bitMaps.get(columnIndex) == null) {
+ return false;
+ }
+ int arrayIndex = rowIndex / ARRAY_SIZE;
+ int elementIndex = rowIndex % ARRAY_SIZE;
+ List<BitMap> columnBitMaps = bitMaps.get(columnIndex);
+ return columnBitMaps.get(arrayIndex).isMarked(elementIndex);
+ }
+
+ public List<List<Object>> getValues() {
+ return values;
+ }
+
+ public List<TSDataType> getTsDataTypes() {
+ return dataTypes;
+ }
+
+ protected void set(int index, long timestamp, int value) {
+ int arrayIndex = index / ARRAY_SIZE;
+ int elementIndex = index % ARRAY_SIZE;
+ timestamps.get(arrayIndex)[elementIndex] = timestamp;
+ indices.get(arrayIndex)[elementIndex] = value;
+ }
+
+ @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+ @Override
+ public VectorTVList clone() {
+ VectorTVList cloneList = new VectorTVList(dataTypes);
+ cloneAs(cloneList);
+ for (int[] indicesArray : indices) {
+ cloneList.indices.add(cloneIndex(indicesArray));
+ }
+ for (int i = 0; i < values.size(); i++) {
+ List<Object> columnValues = values.get(i);
+ for (Object valueArray : columnValues) {
+ cloneList.values.get(i).add(cloneValue(dataTypes.get(i), valueArray));
+ }
+ if (bitMaps != null && bitMaps.get(i) != null) {
+ List<BitMap> columnBitMaps = bitMaps.get(i);
+ if (cloneList.bitMaps == null) {
+ cloneList.bitMaps = new ArrayList<>(dataTypes.size());
+ for (int j = 0; j < dataTypes.size(); j++) {
+ cloneList.bitMaps.add(null);
+ }
+ }
+ if (cloneList.bitMaps.get(i) == null) {
+ List<BitMap> cloneColumnBitMaps = new ArrayList<>();
+ cloneList.bitMaps.set(i, cloneColumnBitMaps);
+ }
+ for (BitMap bitMap : columnBitMaps) {
+ cloneList.bitMaps.get(i).add(cloneBitMap(bitMap));
+ }
+ }
+ }
+ return cloneList;
+ }
+
+ private int[] cloneIndex(int[] array) {
+ int[] cloneArray = new int[array.length];
+ System.arraycopy(array, 0, cloneArray, 0, array.length);
+ return cloneArray;
+ }
+
+ private BitMap cloneBitMap(BitMap bitMap) {
+ byte[] cloneBytes = new byte[bitMap.getByteArray().length];
+ System.arraycopy(bitMap.getByteArray(), 0, cloneBytes, 0, bitMap.getByteArray().length);
+ return new BitMap(bitMap.getSize(), cloneBytes);
+ }
+
+ private Object cloneValue(TSDataType type, Object value) {
+ switch (type) {
+ case TEXT:
+ Binary[] valueT = (Binary[]) value;
+ Binary[] cloneT = new Binary[valueT.length];
+ System.arraycopy(valueT, 0, cloneT, 0, valueT.length);
+ return cloneT;
+ case FLOAT:
+ float[] valueF = (float[]) value;
+ float[] cloneF = new float[valueF.length];
+ System.arraycopy(valueF, 0, cloneF, 0, valueF.length);
+ return cloneF;
+ case INT32:
+ int[] valueI = (int[]) value;
+ int[] cloneI = new int[valueI.length];
+ System.arraycopy(valueI, 0, cloneI, 0, valueI.length);
+ return cloneI;
+ case INT64:
+ long[] valueL = (long[]) value;
+ long[] cloneL = new long[valueL.length];
+ System.arraycopy(valueL, 0, cloneL, 0, valueL.length);
+ return cloneL;
+ case DOUBLE:
+ double[] valueD = (double[]) value;
+ double[] cloneD = new double[valueD.length];
+ System.arraycopy(valueD, 0, cloneD, 0, valueD.length);
+ return cloneD;
+ case BOOLEAN:
+ boolean[] valueB = (boolean[]) value;
+ boolean[] cloneB = new boolean[valueB.length];
+ System.arraycopy(valueB, 0, cloneB, 0, valueB.length);
+ return cloneB;
+ default:
+ return null;
+ }
+ }
+
+ @Override
+ public void sort() {
+ if (sortedTimestamps == null || sortedTimestamps.length < size) {
+ sortedTimestamps =
+ (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, size);
+ }
+ if (sortedIndices == null || sortedIndices.length < size) {
+ sortedIndices = (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, size);
+ }
+ sort(0, size);
+ clearSortedValue();
+ clearSortedTime();
+ sorted = true;
+ }
+
+ @Override
+ void clearValue() {
+ if (indices != null) {
+ for (int[] dataArray : indices) {
+ PrimitiveArrayManager.release(dataArray);
+ }
+ indices.clear();
+ }
+ for (int i = 0; i < dataTypes.size(); i++) {
+ List<Object> columnValues = values.get(i);
+ if (columnValues != null) {
+ for (Object dataArray : columnValues) {
+ PrimitiveArrayManager.release(dataArray);
+ }
+ columnValues.clear();
+ }
+ if (bitMaps != null) {
+ List<BitMap> columnBitMaps = bitMaps.get(i);
+ if (columnBitMaps != null) {
+ columnBitMaps.clear();
+ }
+ }
+ }
+ }
+
+ @Override
+ void clearSortedValue() {
+ if (sortedIndices != null) {
+ sortedIndices = null;
+ }
+ }
+
+ @Override
+ protected void setFromSorted(int src, int dest) {
+ set(
+ dest,
+ sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE],
+ sortedIndices[src / ARRAY_SIZE][src % ARRAY_SIZE]);
+ }
+
+ @Override
+ protected void set(int src, int dest) {
+ long srcT = getTime(src);
+ int srcV = getValueIndex(src);
+ set(dest, srcT, srcV);
+ }
+
+ @Override
+ protected void setToSorted(int src, int dest) {
+ sortedTimestamps[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getTime(src);
+ sortedIndices[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getValueIndex(src);
+ }
+
+ @Override
+ protected void reverseRange(int lo, int hi) {
+ hi--;
+ while (lo < hi) {
+ long loT = getTime(lo);
+ int loV = getValueIndex(lo);
+ long hiT = getTime(hi);
+ int hiV = getValueIndex(hi);
+ set(lo++, hiT, hiV);
+ set(hi--, loT, loV);
+ }
+ }
+
+ @Override
+ protected void expandValues() {
+ indices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32));
+ for (int i = 0; i < dataTypes.size(); i++) {
+ if (bitMaps != null && bitMaps.get(i) != null) {
+ bitMaps.get(i).add(new BitMap(ARRAY_SIZE));
+ }
+ values.get(i).add(getPrimitiveArraysByType(dataTypes.get(i)));
+ }
+ }
+
+ @Override
+ protected void saveAsPivot(int pos) {
+ pivotTime = getTime(pos);
+ pivotIndex = getValueIndex(pos);
+ }
+
+ /* Get the row index value in index column. */
+ public int getValueIndex(int index) {
+ if (index >= size) {
+ throw new ArrayIndexOutOfBoundsException(index);
+ }
+ int arrayIndex = index / ARRAY_SIZE;
+ int elementIndex = index % ARRAY_SIZE;
+ return indices.get(arrayIndex)[elementIndex];
+ }
+
+ @Override
+ protected void setPivotTo(int pos) {
+ set(pos, pivotTime, pivotIndex);
+ }
+
+ @Override
+ public TimeValuePair getTimeValuePair(int index) {
+ if (this.dataTypes.size() == 1) {
+ return new TimeValuePair(getTime(index), ((TsPrimitiveType) getVector(index)).getVector()[0]);
+ } else {
+ return new TimeValuePair(getTime(index), (TsPrimitiveType) getVector(index));
+ }
+ }
+
+ @Override
+ protected TimeValuePair getTimeValuePair(
+ int index, long time, Integer floatPrecision, TSEncoding encoding) {
+ if (this.dataTypes.size() == 1) {
+ return new TimeValuePair(getTime(index), ((TsPrimitiveType) getVector(index)).getVector()[0]);
+ } else {
+ return new TimeValuePair(getTime(index), (TsPrimitiveType) getVector(index));
+ }
+ }
+
+ @Override
+ protected void releaseLastValueArray() {
+ PrimitiveArrayManager.release(indices.remove(indices.size() - 1));
+ for (List<Object> valueList : values) {
+ PrimitiveArrayManager.release(valueList.remove(valueList.size() - 1));
+ }
+ }
+
+ @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+ @Override
+ public void putVectors(long[] time, BitMap[] bitMaps, Object[] value, int start, int end) {
+ checkExpansion();
+ int idx = start;
+
+ updateMinTimeAndSorted(time, start, end);
+
+ while (idx < end) {
+ int inputRemaining = end - idx;
+ int arrayIdx = size / ARRAY_SIZE;
+ int elementIdx = size % ARRAY_SIZE;
+ int internalRemaining = ARRAY_SIZE - elementIdx;
+ if (internalRemaining >= inputRemaining) {
+ // the remaining inputs can fit the last array, copy all remaining inputs into last array
+ System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, inputRemaining);
+ arrayCopy(value, idx, arrayIdx, elementIdx, inputRemaining);
+ for (int i = 0; i < inputRemaining; i++) {
+ indices.get(arrayIdx)[elementIdx + i] = size;
+ if (bitMaps != null) {
+ for (int j = 0; j < bitMaps.length; j++) {
+ if (bitMaps[j] != null && bitMaps[j].isMarked(idx + i)) {
+ markNullValue(j, arrayIdx, elementIdx + i);
+ }
+ }
+ }
+ size++;
+ }
+ break;
+ } else {
+ // the remaining inputs cannot fit the last array, fill the last array and create a new
+ // one and enter the next loop
+ System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, internalRemaining);
+ arrayCopy(value, idx, arrayIdx, elementIdx, internalRemaining);
+ for (int i = 0; i < internalRemaining; i++) {
+ indices.get(arrayIdx)[elementIdx + i] = size;
+ if (bitMaps != null) {
+ for (int j = 0; j < bitMaps.length; j++) {
+ if (bitMaps[j] != null && bitMaps[j].isMarked(idx + i)) {
+ markNullValue(j, arrayIdx, elementIdx + i);
+ }
+ }
+ }
+ size++;
+ }
+ idx += internalRemaining;
+ checkExpansion();
+ }
+ }
+ }
+
+ private void arrayCopy(Object[] value, int idx, int arrayIndex, int elementIndex, int remaining) {
+ for (int i = 0; i < values.size(); i++) {
+ List<Object> columnValues = values.get(i);
+ switch (dataTypes.get(i)) {
+ case TEXT:
+ Binary[] arrayT = ((Binary[]) columnValues.get(arrayIndex));
+ System.arraycopy(value[i], idx, arrayT, elementIndex, remaining);
+ break;
+ case FLOAT:
+ float[] arrayF = ((float[]) columnValues.get(arrayIndex));
+ System.arraycopy(value[i], idx, arrayF, elementIndex, remaining);
+ break;
+ case INT32:
+ int[] arrayI = ((int[]) columnValues.get(arrayIndex));
+ System.arraycopy(value[i], idx, arrayI, elementIndex, remaining);
+ break;
+ case INT64:
+ long[] arrayL = ((long[]) columnValues.get(arrayIndex));
+ System.arraycopy(value[i], idx, arrayL, elementIndex, remaining);
+ break;
+ case DOUBLE:
+ double[] arrayD = ((double[]) columnValues.get(arrayIndex));
+ System.arraycopy(value[i], idx, arrayD, elementIndex, remaining);
+ break;
+ case BOOLEAN:
+ boolean[] arrayB = ((boolean[]) columnValues.get(arrayIndex));
+ System.arraycopy(value[i], idx, arrayB, elementIndex, remaining);
+ break;
+ default:
+ break;
+ }
+ }
+ }
+
+ private void markNullValue(int columnIndex, int arrayIndex, int elementIndex) {
+ // init BitMaps if doesn't have
+ if (bitMaps == null) {
+ bitMaps = new ArrayList<>(dataTypes.size());
+ for (int i = 0; i < dataTypes.size(); i++) {
+ bitMaps.add(null);
+ }
+ }
+
+ // if the bitmap in columnIndex is null, init the bitmap of this column from the beginning
+ if (bitMaps.get(columnIndex) == null) {
+ List<BitMap> columnBitMaps = new ArrayList<>();
+ for (int i = 0; i < values.get(columnIndex).size(); i++) {
+ columnBitMaps.add(new BitMap(ARRAY_SIZE));
+ }
+ bitMaps.set(columnIndex, columnBitMaps);
+ }
+
+ // mark the null value in the current bitmap
+ bitMaps.get(columnIndex).get(arrayIndex).mark(elementIndex);
+ }
+
+ @Override
+ public TSDataType getDataType() {
+ return TSDataType.VECTOR;
+ }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/recover/LogReplayer.java b/server/src/main/java/org/apache/iotdb/db/writelog/recover/LogReplayer.java
index 4589ba8..e9a6e5b 100644
--- a/server/src/main/java/org/apache/iotdb/db/writelog/recover/LogReplayer.java
+++ b/server/src/main/java/org/apache/iotdb/db/writelog/recover/LogReplayer.java
@@ -40,6 +40,7 @@
import org.apache.iotdb.db.writelog.io.ILogReader;
import org.apache.iotdb.db.writelog.manager.MultiFileLogNodeManager;
import org.apache.iotdb.db.writelog.node.WriteLogNode;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
import org.slf4j.Logger;
@@ -187,6 +188,7 @@
}
private void checkDataTypeAndMarkFailed(final MeasurementMNode[] mNodes, InsertPlan tPlan) {
+ int columnIndex = 0;
for (int i = 0; i < mNodes.length; i++) {
if (mNodes[i] == null) {
tPlan.markFailedMeasurementInsertion(
@@ -195,11 +197,32 @@
tPlan.getDeviceId().getFullPath()
+ IoTDBConstant.PATH_SEPARATOR
+ tPlan.getMeasurements()[i]));
- } else if (mNodes[i].getSchema().getType() != tPlan.getDataTypes()[i]) {
+ columnIndex++;
+ } else if (mNodes[i].getSchema().getType() == TSDataType.VECTOR) {
+ List<TSDataType> datatypes = mNodes[i].getSchema().getValueTSDataTypeList();
+ for (int j = 0; j < datatypes.size(); j++) {
+ if (tPlan.getDataTypes()[columnIndex] == null) {
+ tPlan.getDataTypes()[columnIndex] = datatypes.get(j);
+ } else if (datatypes.get(j) != tPlan.getDataTypes()[columnIndex]) {
+ tPlan.markFailedMeasurementInsertion(
+ i,
+ new DataTypeMismatchException(
+ mNodes[i].getSchema().getValueMeasurementIdList().get(j),
+ tPlan.getDataTypes()[columnIndex],
+ datatypes.get(j)));
+ }
+ columnIndex++;
+ }
+ } else if (mNodes[i].getSchema().getType() != tPlan.getDataTypes()[columnIndex]) {
tPlan.markFailedMeasurementInsertion(
i,
new DataTypeMismatchException(
- mNodes[i].getName(), tPlan.getDataTypes()[i], mNodes[i].getSchema().getType()));
+ mNodes[i].getName(),
+ tPlan.getDataTypes()[columnIndex],
+ mNodes[i].getSchema().getType()));
+ columnIndex++;
+ } else {
+ columnIndex++;
}
}
}
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableFlushTaskTest.java b/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableFlushTaskTest.java
index a1b5d55..af52254 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableFlushTaskTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableFlushTaskTest.java
@@ -21,8 +21,10 @@
import org.apache.iotdb.db.constant.TestConstant;
import org.apache.iotdb.db.engine.MetadataManagerHelper;
import org.apache.iotdb.db.engine.flush.MemTableFlushTask;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.utils.EnvironmentUtils;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
@@ -30,6 +32,7 @@
import org.junit.Before;
import org.junit.Test;
+import java.io.IOException;
import java.util.concurrent.ExecutionException;
import static org.junit.Assert.assertEquals;
@@ -100,4 +103,58 @@
assertEquals(MemTableTestUtils.dataType0, chunkMetaData.getDataType());
assertEquals(endTime - startTime + 1, chunkMetaData.getNumOfPoints());
}
+
+ @Test
+ public void testFlushVectorMemTable()
+ throws ExecutionException, InterruptedException, IllegalPathException, IOException {
+ MemTableTestUtils.produceVectorData(memTable);
+ MemTableFlushTask memTableFlushTask = new MemTableFlushTask(memTable, writer, storageGroup);
+ assertTrue(
+ writer
+ .getVisibleMetadataList(MemTableTestUtils.deviceId0, "sensor0", TSDataType.BOOLEAN)
+ .isEmpty());
+ memTableFlushTask.syncFlushMemTable();
+ writer.makeMetadataVisible();
+ assertEquals(
+ 1,
+ writer
+ .getVisibleMetadataList(MemTableTestUtils.deviceId0, "sensor0", TSDataType.BOOLEAN)
+ .size());
+ ChunkMetadata chunkMetaData =
+ writer
+ .getVisibleMetadataList(MemTableTestUtils.deviceId0, "sensor0", TSDataType.BOOLEAN)
+ .get(0);
+ assertEquals("sensor0", chunkMetaData.getMeasurementUid());
+ assertEquals(startTime, chunkMetaData.getStartTime());
+ assertEquals(endTime, chunkMetaData.getEndTime());
+ assertEquals(TSDataType.BOOLEAN, chunkMetaData.getDataType());
+ assertEquals(endTime - startTime + 1, chunkMetaData.getNumOfPoints());
+ }
+
+ @Test
+ public void testFlushNullableVectorMemTable()
+ throws ExecutionException, InterruptedException, IllegalPathException, IOException {
+ MemTableTestUtils.produceNullableVectorData(memTable);
+ MemTableFlushTask memTableFlushTask = new MemTableFlushTask(memTable, writer, storageGroup);
+ assertTrue(
+ writer
+ .getVisibleMetadataList(MemTableTestUtils.deviceId0, "sensor0", TSDataType.BOOLEAN)
+ .isEmpty());
+ memTableFlushTask.syncFlushMemTable();
+ writer.makeMetadataVisible();
+ assertEquals(
+ 1,
+ writer
+ .getVisibleMetadataList(MemTableTestUtils.deviceId0, "sensor0", TSDataType.BOOLEAN)
+ .size());
+ ChunkMetadata chunkMetaData =
+ writer
+ .getVisibleMetadataList(MemTableTestUtils.deviceId0, "sensor0", TSDataType.BOOLEAN)
+ .get(0);
+ assertEquals("sensor0", chunkMetaData.getMeasurementUid());
+ assertEquals(startTime, chunkMetaData.getStartTime());
+ assertEquals(endTime, chunkMetaData.getEndTime());
+ assertEquals(TSDataType.BOOLEAN, chunkMetaData.getDataType());
+ assertEquals(endTime - startTime + 1, chunkMetaData.getNumOfPoints());
+ }
}
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableTestUtils.java b/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableTestUtils.java
index afb2589..05b6fbb 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableTestUtils.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableTestUtils.java
@@ -18,11 +18,22 @@
*/
package org.apache.iotdb.db.engine.memtable;
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.utils.BitMap;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.Schema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
+
+import java.util.ArrayList;
+import java.util.List;
public class MemTableTestUtils {
@@ -51,14 +62,74 @@
}
for (long l = startTime; l <= endTime; l++) {
iMemTable.write(
- deviceId,
- measurementId,
- new MeasurementSchema(measurementId, dataType, TSEncoding.PLAIN),
- l,
- (int) l);
+ deviceId, new MeasurementSchema(measurementId, dataType, TSEncoding.PLAIN), l, (int) l);
}
}
+ public static void produceVectorData(IMemTable iMemTable) throws IllegalPathException {
+ iMemTable.write(genInsertTablePlan(), 1, 101);
+ }
+
+ private static InsertTabletPlan genInsertTablePlan() throws IllegalPathException {
+ String[] measurements = new String[2];
+ measurements[0] = "sensor0";
+ measurements[1] = "sensor1";
+
+ List<Integer> dataTypesList = new ArrayList<>();
+ TSDataType[] dataTypes = new TSDataType[2];
+ dataTypesList.add(TSDataType.BOOLEAN.ordinal());
+ dataTypesList.add(TSDataType.INT64.ordinal());
+ dataTypes[0] = TSDataType.BOOLEAN;
+ dataTypes[1] = TSDataType.INT64;
+
+ TSEncoding[] encodings = new TSEncoding[2];
+ encodings[0] = TSEncoding.PLAIN;
+ encodings[1] = TSEncoding.GORILLA;
+
+ MeasurementMNode[] mNodes = new MeasurementMNode[2];
+ IMeasurementSchema schema =
+ new VectorMeasurementSchema(
+ IoTDBConstant.ALIGN_TIMESERIES_PREFIX, measurements, dataTypes, encodings);
+ mNodes[0] = new MeasurementMNode(null, "sensor0", schema, null);
+ mNodes[1] = new MeasurementMNode(null, "sensor1", schema, null);
+
+ InsertTabletPlan insertTabletPlan =
+ new InsertTabletPlan(
+ new PartialPath(deviceId0), new String[] {"(sensor0,sensor1)"}, dataTypesList);
+
+ long[] times = new long[101];
+ Object[] columns = new Object[2];
+ columns[0] = new boolean[101];
+ columns[1] = new long[101];
+
+ for (long r = 0; r < 101; r++) {
+ times[(int) r] = r;
+ ((boolean[]) columns[0])[(int) r] = false;
+ ((long[]) columns[1])[(int) r] = r;
+ }
+ insertTabletPlan.setTimes(times);
+ insertTabletPlan.setColumns(columns);
+ insertTabletPlan.setRowCount(times.length);
+ insertTabletPlan.setMeasurementMNodes(mNodes);
+ insertTabletPlan.setStart(0);
+ insertTabletPlan.setEnd(100);
+
+ return insertTabletPlan;
+ }
+
+ public static void produceNullableVectorData(IMemTable iMemTable) throws IllegalPathException {
+ InsertTabletPlan plan = genInsertTablePlan();
+ BitMap[] bitMaps = new BitMap[2];
+ bitMaps[1] = new BitMap(101);
+ for (int r = 0; r < 101; r++) {
+ if (r % 2 == 1) {
+ bitMaps[1].mark(r);
+ }
+ }
+ plan.setBitMaps(bitMaps);
+ iMemTable.write(plan, 1, 101);
+ }
+
public static Schema getSchema() {
return schema;
}
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemtableBenchmark.java b/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemtableBenchmark.java
index 0e0951f..eea2c7a 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemtableBenchmark.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemtableBenchmark.java
@@ -46,7 +46,6 @@
for (int j = 0; j < numOfMeasurement; j++) {
memTable.write(
deviceId,
- measurementId[j],
new MeasurementSchema(measurementId[j], tsDataType, TSEncoding.PLAIN),
System.nanoTime(),
String.valueOf(System.currentTimeMillis()));
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTableTest.java b/server/src/test/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTableTest.java
index 907beb2..330db67 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTableTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTableTest.java
@@ -18,28 +18,38 @@
*/
package org.apache.iotdb.db.engine.memtable;
+import org.apache.iotdb.db.conf.IoTDBConstant;
import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.exception.metadata.MetadataException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
import org.apache.iotdb.db.utils.MathUtils;
import org.apache.iotdb.db.utils.datastructure.TVList;
import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+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.read.reader.IPointReader;
import org.apache.iotdb.tsfile.utils.Binary;
import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
+import java.util.List;
import java.util.Random;
public class PrimitiveMemTableTest {
@@ -61,7 +71,7 @@
for (int i = 0; i < count; i++) {
series.write(i, i);
}
- IPointReader it = series.getSortedTVListForQuery().getIterator();
+ IPointReader it = series.getSortedTvListForQuery().getIterator();
int i = 0;
while (it.hasNextTimeValuePair()) {
Assert.assertEquals(i, it.nextTimeValuePair().getTimestamp());
@@ -84,7 +94,6 @@
for (int i = 0; i < dataSize; i++) {
memTable.write(
deviceId,
- measurementId[0],
new MeasurementSchema(measurementId[0], TSDataType.INT32, TSEncoding.PLAIN),
dataSize - i - 1,
i + 10);
@@ -92,7 +101,6 @@
for (int i = 0; i < dataSize; i++) {
memTable.write(
deviceId,
- measurementId[0],
new MeasurementSchema(measurementId[0], TSDataType.INT32, TSEncoding.PLAIN),
i,
i);
@@ -101,9 +109,12 @@
memTable.query(
deviceId,
measurementId[0],
- TSDataType.INT32,
- TSEncoding.RLE,
- Collections.emptyMap(),
+ new MeasurementSchema(
+ measurementId[0],
+ TSDataType.INT32,
+ TSEncoding.RLE,
+ CompressionType.UNCOMPRESSED,
+ Collections.emptyMap()),
Long.MIN_VALUE,
null);
IPointReader iterator = memChunk.getPointReader();
@@ -128,7 +139,6 @@
for (TimeValuePair aRet : ret) {
memTable.write(
deviceId,
- sensorId,
new MeasurementSchema(sensorId, dataType, encoding),
aRet.getTimestamp(),
aRet.getValue().getValue());
@@ -138,9 +148,12 @@
.query(
deviceId,
sensorId,
- dataType,
- encoding,
- Collections.emptyMap(),
+ new MeasurementSchema(
+ sensorId,
+ dataType,
+ encoding,
+ CompressionType.UNCOMPRESSED,
+ Collections.emptyMap()),
Long.MIN_VALUE,
null)
.getPointReader();
@@ -173,6 +186,53 @@
}
}
+ private void writeVector(IMemTable memTable)
+ throws IOException, QueryProcessException, MetadataException {
+ memTable.write(genInsertTablePlan(), 0, 100);
+
+ IPointReader tvPair =
+ memTable
+ .query(
+ "root.sg.device5",
+ "sensor1",
+ new VectorMeasurementSchema(
+ IoTDBConstant.ALIGN_TIMESERIES_PREFIX + 0,
+ new String[] {"sensor1"},
+ new TSDataType[] {TSDataType.INT64},
+ new TSEncoding[] {TSEncoding.GORILLA},
+ CompressionType.UNCOMPRESSED),
+ Long.MIN_VALUE,
+ null)
+ .getPointReader();
+ for (int i = 0; i < 100; i++) {
+ tvPair.hasNextTimeValuePair();
+ TimeValuePair next = tvPair.nextTimeValuePair();
+ Assert.assertEquals(i, next.getTimestamp());
+ Assert.assertEquals(i, next.getValue().getLong());
+ }
+
+ tvPair =
+ memTable
+ .query(
+ "root.sg.device5",
+ "$#$1",
+ new VectorMeasurementSchema(
+ IoTDBConstant.ALIGN_TIMESERIES_PREFIX + 0,
+ new String[] {"sensor0", "sensor1"},
+ new TSDataType[] {TSDataType.BOOLEAN, TSDataType.INT64},
+ new TSEncoding[] {TSEncoding.PLAIN, TSEncoding.GORILLA},
+ CompressionType.UNCOMPRESSED),
+ Long.MIN_VALUE,
+ null)
+ .getPointReader();
+ for (int i = 0; i < 100; i++) {
+ tvPair.hasNextTimeValuePair();
+ TimeValuePair next = tvPair.nextTimeValuePair();
+ Assert.assertEquals(i, next.getTimestamp());
+ Assert.assertEquals(i, next.getValue().getVector()[1].getLong());
+ }
+ }
+
@Test
public void testFloatType() throws IOException, QueryProcessException, MetadataException {
IMemTable memTable = new PrimitiveMemTable();
@@ -199,6 +259,7 @@
write(memTable, deviceId, measurementId[index++], TSDataType.FLOAT, TSEncoding.RLE, size);
write(memTable, deviceId, measurementId[index++], TSDataType.DOUBLE, TSEncoding.RLE, size);
write(memTable, deviceId, measurementId[index++], TSDataType.TEXT, TSEncoding.PLAIN, size);
+ writeVector(memTable);
}
private TimeValuePair[] genTimeValuePair(int size, TSDataType dataType) {
@@ -241,4 +302,53 @@
}
return ret;
}
+
+ private InsertTabletPlan genInsertTablePlan() throws IllegalPathException {
+ String[] measurements = new String[2];
+ measurements[0] = "sensor0";
+ measurements[1] = "sensor1";
+
+ List<Integer> dataTypesList = new ArrayList<>();
+ TSDataType[] dataTypes = new TSDataType[2];
+ dataTypesList.add(TSDataType.BOOLEAN.ordinal());
+ dataTypesList.add(TSDataType.INT64.ordinal());
+ dataTypes[0] = TSDataType.BOOLEAN;
+ dataTypes[1] = TSDataType.INT64;
+
+ TSEncoding[] encodings = new TSEncoding[2];
+ encodings[0] = TSEncoding.PLAIN;
+ encodings[1] = TSEncoding.GORILLA;
+
+ String deviceId = "root.sg.device5";
+
+ MeasurementMNode[] mNodes = new MeasurementMNode[2];
+ IMeasurementSchema schema =
+ new VectorMeasurementSchema(
+ IoTDBConstant.ALIGN_TIMESERIES_PREFIX + 0, measurements, dataTypes, encodings);
+ mNodes[0] = new MeasurementMNode(null, "sensor0", schema, null);
+ mNodes[1] = new MeasurementMNode(null, "sensor1", schema, null);
+
+ InsertTabletPlan insertTabletPlan =
+ new InsertTabletPlan(
+ new PartialPath(deviceId), new String[] {"(sensor0,sensor1)"}, dataTypesList);
+
+ long[] times = new long[100];
+ Object[] columns = new Object[2];
+ columns[0] = new boolean[100];
+ columns[1] = new long[100];
+
+ for (long r = 0; r < 100; r++) {
+ times[(int) r] = r;
+ ((boolean[]) columns[0])[(int) r] = false;
+ ((long[]) columns[1])[(int) r] = r;
+ }
+ insertTabletPlan.setTimes(times);
+ insertTabletPlan.setColumns(columns);
+ insertTabletPlan.setRowCount(times.length);
+ insertTabletPlan.setMeasurementMNodes(mNodes);
+ insertTabletPlan.setStart(0);
+ insertTabletPlan.setEnd(100);
+
+ return insertTabletPlan;
+ }
}
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java b/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java
index fa82917..7375045 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java
@@ -124,7 +124,7 @@
RawDataQueryPlan queryPlan = new RawDataQueryPlan();
queryPlan.setDeduplicatedDataTypes(dataTypes);
- queryPlan.setDeduplicatedPaths(pathList);
+ queryPlan.setDeduplicatedPathsAndUpdate(pathList);
QueryDataSet dataSet = router.rawDataQuery(queryPlan, TEST_QUERY_CONTEXT);
int count = 0;
@@ -163,7 +163,7 @@
RawDataQueryPlan queryPlan = new RawDataQueryPlan();
queryPlan.setDeduplicatedDataTypes(dataTypes);
- queryPlan.setDeduplicatedPaths(pathList);
+ queryPlan.setDeduplicatedPathsAndUpdate(pathList);
QueryDataSet dataSet = router.rawDataQuery(queryPlan, TEST_QUERY_CONTEXT);
int count = 0;
@@ -212,7 +212,7 @@
RawDataQueryPlan queryPlan = new RawDataQueryPlan();
queryPlan.setDeduplicatedDataTypes(dataTypes);
- queryPlan.setDeduplicatedPaths(pathList);
+ queryPlan.setDeduplicatedPathsAndUpdate(pathList);
QueryDataSet dataSet = router.rawDataQuery(queryPlan, TEST_QUERY_CONTEXT);
int count = 0;
@@ -262,7 +262,7 @@
RawDataQueryPlan queryPlan = new RawDataQueryPlan();
queryPlan.setDeduplicatedDataTypes(dataTypes);
- queryPlan.setDeduplicatedPaths(pathList);
+ queryPlan.setDeduplicatedPathsAndUpdate(pathList);
QueryDataSet dataSet = router.rawDataQuery(queryPlan, TEST_QUERY_CONTEXT);
int count = 0;
@@ -333,7 +333,7 @@
RawDataQueryPlan queryPlan = new RawDataQueryPlan();
queryPlan.setDeduplicatedDataTypes(dataTypes);
- queryPlan.setDeduplicatedPaths(pathList);
+ queryPlan.setDeduplicatedPathsAndUpdate(pathList);
QueryDataSet dataSet = router.rawDataQuery(queryPlan, TEST_QUERY_CONTEXT);
int count = 0;
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
index 56bd9db..3d7e2b6 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
@@ -39,6 +39,7 @@
import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
import org.apache.iotdb.db.query.context.QueryContext;
import org.apache.iotdb.db.utils.EnvironmentUtils;
+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;
@@ -126,9 +127,12 @@
tsfileProcessor.query(
deviceId,
measurementId,
- TSDataType.INT32,
- TSEncoding.RLE,
- Collections.emptyMap(),
+ new MeasurementSchema(
+ measurementId,
+ TSDataType.INT32,
+ TSEncoding.RLE,
+ CompressionType.UNCOMPRESSED,
+ Collections.emptyMap()),
new QueryContext(),
tsfileResourcesForQuery);
}
@@ -164,7 +168,7 @@
}
processor.syncCloseAllWorkingTsFileProcessors();
QueryDataSource queryDataSource =
- processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+ processor.query(new PartialPath(deviceId, measurementId), context, null, null);
Assert.assertEquals(10, queryDataSource.getSeqResources().size());
for (TsFileResource resource : queryDataSource.getSeqResources()) {
Assert.assertTrue(resource.isClosed());
@@ -194,7 +198,7 @@
processor.syncCloseAllWorkingTsFileProcessors();
QueryDataSource queryDataSource =
- processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+ processor.query(new PartialPath(deviceId, measurementId), context, null, null);
Assert.assertEquals(0, queryDataSource.getUnseqResources().size());
}
@@ -255,7 +259,7 @@
processor.syncCloseAllWorkingTsFileProcessors();
QueryDataSource queryDataSource =
- processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+ processor.query(new PartialPath(deviceId, measurementId), context, null, null);
Assert.assertEquals(2, queryDataSource.getSeqResources().size());
Assert.assertEquals(1, queryDataSource.getUnseqResources().size());
@@ -286,7 +290,7 @@
processor.syncCloseAllWorkingTsFileProcessors();
QueryDataSource queryDataSource =
- processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+ processor.query(new PartialPath(deviceId, measurementId), context, null, null);
Assert.assertEquals(10, queryDataSource.getSeqResources().size());
Assert.assertEquals(10, queryDataSource.getUnseqResources().size());
for (TsFileResource resource : queryDataSource.getSeqResources()) {
@@ -327,7 +331,7 @@
}
QueryDataSource queryDataSource =
- processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+ processor.query(new PartialPath(deviceId, measurementId), context, null, null);
Assert.assertEquals(10, queryDataSource.getSeqResources().size());
Assert.assertEquals(0, queryDataSource.getUnseqResources().size());
for (TsFileResource resource : queryDataSource.getSeqResources()) {
@@ -409,7 +413,7 @@
}
QueryDataSource queryDataSource =
- processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+ processor.query(new PartialPath(deviceId, measurementId), context, null, null);
Assert.assertEquals(2, queryDataSource.getSeqResources().size());
Assert.assertEquals(0, queryDataSource.getUnseqResources().size());
@@ -491,7 +495,7 @@
}
QueryDataSource queryDataSource =
- processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+ processor.query(new PartialPath(deviceId, measurementId), context, null, null);
Assert.assertEquals(2, queryDataSource.getSeqResources().size());
Assert.assertEquals(0, queryDataSource.getUnseqResources().size());
@@ -573,7 +577,7 @@
}
QueryDataSource queryDataSource =
- processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+ processor.query(new PartialPath(deviceId, measurementId), context, null, null);
Assert.assertEquals(2, queryDataSource.getSeqResources().size());
Assert.assertEquals(0, queryDataSource.getUnseqResources().size());
@@ -612,7 +616,7 @@
}
QueryDataSource queryDataSource =
- processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+ processor.query(new PartialPath(deviceId, measurementId), context, null, null);
Assert.assertEquals(10, queryDataSource.getSeqResources().size());
for (TsFileResource resource : queryDataSource.getSeqResources()) {
Assert.assertTrue(resource.isClosed());
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java
index f0e8fe3..f08db84 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java
@@ -226,7 +226,7 @@
// files before ttl
QueryDataSource dataSource =
storageGroupProcessor.query(
- new PartialPath(sg1), s1, EnvironmentUtils.TEST_QUERY_CONTEXT, null, null);
+ new PartialPath(sg1, s1), EnvironmentUtils.TEST_QUERY_CONTEXT, null, null);
List<TsFileResource> seqResource = dataSource.getSeqResources();
List<TsFileResource> unseqResource = dataSource.getUnseqResources();
assertEquals(4, seqResource.size());
@@ -237,7 +237,7 @@
// files after ttl
dataSource =
storageGroupProcessor.query(
- new PartialPath(sg1), s1, EnvironmentUtils.TEST_QUERY_CONTEXT, null, null);
+ new PartialPath(sg1, s1), EnvironmentUtils.TEST_QUERY_CONTEXT, null, null);
seqResource = dataSource.getSeqResources();
unseqResource = dataSource.getUnseqResources();
assertTrue(seqResource.size() < 4);
@@ -272,7 +272,7 @@
storageGroupProcessor.setDataTTL(0);
dataSource =
storageGroupProcessor.query(
- new PartialPath(sg1), s1, EnvironmentUtils.TEST_QUERY_CONTEXT, null, null);
+ new PartialPath(sg1, s1), EnvironmentUtils.TEST_QUERY_CONTEXT, null, null);
seqResource = dataSource.getSeqResources();
unseqResource = dataSource.getUnseqResources();
assertEquals(0, seqResource.size());
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
index f972a83..5f43331 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
@@ -30,12 +30,14 @@
import org.apache.iotdb.db.rescon.SystemInfo;
import org.apache.iotdb.db.utils.EnvironmentUtils;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+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.read.reader.IPointReader;
import org.apache.iotdb.tsfile.write.record.TSRecord;
import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
import org.junit.After;
@@ -104,7 +106,12 @@
SystemInfo.getInstance().reportStorageGroupStatus(sgInfo);
List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
processor.query(
- deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+ deviceId,
+ measurementId,
+ new MeasurementSchema(
+ measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
+ context,
+ tsfileResourcesForQuery);
assertTrue(tsfileResourcesForQuery.isEmpty());
for (int i = 1; i <= 100; i++) {
@@ -116,7 +123,12 @@
// query data in memory
tsfileResourcesForQuery.clear();
processor.query(
- deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+ deviceId,
+ measurementId,
+ new MeasurementSchema(
+ measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
+ context,
+ tsfileResourcesForQuery);
assertFalse(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
List<ReadOnlyMemChunk> memChunks = tsfileResourcesForQuery.get(0).getReadOnlyMemChunk();
for (ReadOnlyMemChunk chunk : memChunks) {
@@ -134,7 +146,12 @@
tsfileResourcesForQuery.clear();
processor.query(
- deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+ deviceId,
+ measurementId,
+ new MeasurementSchema(
+ measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
+ context,
+ tsfileResourcesForQuery);
assertTrue(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
assertEquals(1, tsfileResourcesForQuery.get(0).getChunkMetadataList().size());
assertEquals(
@@ -166,7 +183,12 @@
SystemInfo.getInstance().reportStorageGroupStatus(sgInfo);
List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
processor.query(
- deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+ deviceId,
+ measurementId,
+ new MeasurementSchema(
+ measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
+ context,
+ tsfileResourcesForQuery);
assertTrue(tsfileResourcesForQuery.isEmpty());
for (int i = 1; i <= 100; i++) {
@@ -178,7 +200,12 @@
// query data in memory
tsfileResourcesForQuery.clear();
processor.query(
- deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+ deviceId,
+ measurementId,
+ new MeasurementSchema(
+ measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
+ context,
+ tsfileResourcesForQuery);
assertFalse(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
int num = 1;
List<ReadOnlyMemChunk> memChunks = tsfileResourcesForQuery.get(0).getReadOnlyMemChunk();
@@ -197,7 +224,12 @@
tsfileResourcesForQuery.clear();
processor.query(
- deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+ deviceId,
+ measurementId,
+ new MeasurementSchema(
+ measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
+ context,
+ tsfileResourcesForQuery);
assertTrue(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
assertEquals(1, tsfileResourcesForQuery.get(0).getChunkMetadataList().size());
assertEquals(
@@ -254,7 +286,12 @@
SystemInfo.getInstance().reportStorageGroupStatus(sgInfo);
List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
processor.query(
- deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+ deviceId,
+ measurementId,
+ new MeasurementSchema(
+ measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
+ context,
+ tsfileResourcesForQuery);
assertTrue(tsfileResourcesForQuery.isEmpty());
for (int flushId = 0; flushId < 10; flushId++) {
@@ -269,7 +306,12 @@
tsfileResourcesForQuery.clear();
processor.query(
- deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+ deviceId,
+ measurementId,
+ new MeasurementSchema(
+ measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
+ context,
+ tsfileResourcesForQuery);
assertFalse(tsfileResourcesForQuery.isEmpty());
assertTrue(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
assertEquals(10, tsfileResourcesForQuery.get(0).getChunkMetadataList().size());
@@ -302,7 +344,12 @@
List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
processor.query(
- deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+ deviceId,
+ measurementId,
+ new MeasurementSchema(
+ measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
+ context,
+ tsfileResourcesForQuery);
assertTrue(tsfileResourcesForQuery.isEmpty());
for (int i = 1; i <= 100; i++) {
@@ -314,7 +361,12 @@
// query data in memory
tsfileResourcesForQuery.clear();
processor.query(
- deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+ deviceId,
+ measurementId,
+ new MeasurementSchema(
+ measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
+ context,
+ tsfileResourcesForQuery);
assertFalse(tsfileResourcesForQuery.isEmpty());
assertFalse(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
List<ReadOnlyMemChunk> memChunks = tsfileResourcesForQuery.get(0).getReadOnlyMemChunk();
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IOTDBInsertAlignedValuesIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IOTDBInsertAlignedValuesIT.java
new file mode 100644
index 0000000..fadfd7f
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IOTDBInsertAlignedValuesIT.java
@@ -0,0 +1,149 @@
+/*
+ * 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.integration;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Objects;
+
+public class IOTDBInsertAlignedValuesIT {
+ private static Connection connection;
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ EnvironmentUtils.closeStatMonitor();
+ EnvironmentUtils.envSetUp();
+ IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(true);
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ connection =
+ DriverManager.getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ close();
+ EnvironmentUtils.cleanEnv();
+ }
+
+ private static void close() {
+ if (Objects.nonNull(connection)) {
+ try {
+ connection.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+ @Test
+ public void testInsertAlignedValues() throws SQLException {
+ Statement st0 = connection.createStatement();
+ st0.execute(
+ "insert into root.t1.wf01.wt01(time, (status, temperature)) values (4000, (true, 17.1))");
+ st0.execute(
+ "insert into root.t1.wf01.wt01(time, (status, temperature)) values (5000, (true, 20.1))");
+ st0.execute(
+ "insert into root.t1.wf01.wt01(time, (status, temperature)) values (6000, (true, 22))");
+ st0.close();
+
+ Statement st1 = connection.createStatement();
+
+ ResultSet rs1 = st1.executeQuery("select status from root.t1.wf01.wt01");
+ rs1.next();
+ Assert.assertEquals(true, rs1.getBoolean(2));
+
+ ResultSet rs2 = st1.executeQuery("select * from root.t1.wf01.wt01");
+ rs2.next();
+ Assert.assertEquals(4000, rs2.getLong(1));
+ Assert.assertEquals(true, rs2.getBoolean(2));
+ Assert.assertEquals(17.1, rs2.getFloat(3), 0.1);
+
+ rs2.next();
+ Assert.assertEquals(5000, rs2.getLong(1));
+ Assert.assertEquals(true, rs2.getBoolean(2));
+ Assert.assertEquals(20.1, rs2.getFloat(3), 0.1);
+
+ rs2.next();
+ Assert.assertEquals(6000, rs2.getLong(1));
+ Assert.assertEquals(true, rs2.getBoolean(2));
+ Assert.assertEquals(22, rs2.getFloat(3), 0.1);
+ st1.close();
+ }
+
+ @Test
+ public void testInsertAlignedNullableValues() throws SQLException {
+ Statement st0 = connection.createStatement();
+ st0.execute(
+ "insert into root.t1.wf01.wt01(time, (status, temperature)) values (4000, (true, 17.1))");
+ st0.execute(
+ "insert into root.t1.wf01.wt01(time, (status, temperature)) values (5000, (true, null))");
+ st0.execute(
+ "insert into root.t1.wf01.wt01(time, (status, temperature)) values (6000, (NULL, 22))");
+ st0.close();
+
+ Statement st1 = connection.createStatement();
+
+ ResultSet rs1 = st1.executeQuery("select status from root.t1.wf01.wt01");
+ rs1.next();
+ Assert.assertEquals(true, rs1.getBoolean(2));
+
+ ResultSet rs2 = st1.executeQuery("select * from root.t1.wf01.wt01");
+ rs2.next();
+ Assert.assertEquals(4000, rs2.getLong(1));
+ Assert.assertEquals(true, rs2.getBoolean(2));
+ Assert.assertEquals(17.1, rs2.getFloat(3), 0.1);
+
+ rs2.next();
+ Assert.assertEquals(5000, rs2.getLong(1));
+ Assert.assertEquals(true, rs2.getObject(2));
+ Assert.assertEquals(null, rs2.getObject(3));
+
+ rs2.next();
+ Assert.assertEquals(6000, rs2.getLong(1));
+ Assert.assertEquals(null, rs2.getObject(2));
+ Assert.assertEquals(22.0f, rs2.getObject(3));
+ st1.close();
+ }
+
+ @Test(expected = Exception.class)
+ public void testInsertWithWrongMeasurementNum1() throws SQLException {
+ Statement st1 = connection.createStatement();
+ st1.execute("insert into root.t1.wf01.wt01(time, (status, temperature)) values(11000, 100)");
+ }
+
+ @Test(expected = Exception.class)
+ public void testInsertWithWrongMeasurementNum2() throws SQLException {
+ Statement st1 = connection.createStatement();
+ st1.execute(
+ "insert into root.t1.wf01.wt01(time, (status, temperature)) values(11000, (100, 300, 400))");
+ }
+}
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IOTDBInsertIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IOTDBInsertIT.java
index 0ffa952..4d8c4dc 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IOTDBInsertIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IOTDBInsertIT.java
@@ -91,7 +91,7 @@
}
@Test
- public void testNormalInset() throws SQLException {
+ public void testNormalInsert() throws SQLException {
Statement st0 = connection.createStatement();
st0.execute("insert into root.t1.wf01.wt01(timestamp, status) values (1000, true)");
st0.execute("insert into root.t1.wf01.wt01(timestamp, status) values (2000, false)");
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCompressTypeIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCompressTypeIT.java
index c04a4e9..05dc1af 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCompressTypeIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBCompressTypeIT.java
@@ -32,11 +32,6 @@
import java.sql.ResultSet;
import java.sql.Statement;
-/**
- * @mail yuqi4733@gmail.com
- * @description your description
- * @time 14/12/20 上午11:05
- */
public class IoTDBCompressTypeIT {
@Before
public void setUp() {
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java
index bd402a4..088cdba 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java
@@ -326,10 +326,18 @@
Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
Statement statement = connection.createStatement()) {
statement.execute(
- "CREATE TIMESERIES root.ln.d1.\"status,01\" WITH DATATYPE=BOOLEAN," + " ENCODING=PLAIN");
- statement.execute("INSERT INTO root.ln.d1(timestamp,\"status,01\") " + "values(300,true)");
+ "CREATE TIMESERIES root.ln.d1.\"status,01\" WITH DATATYPE=BOOLEAN, ENCODING=PLAIN");
+ statement.execute("INSERT INTO root.ln.d1(timestamp,\"status,01\") VALUES(300, true)");
statement.execute("INSERT INTO root.ln.d1(timestamp,\"status,01\") VALUES(500, false)");
+ try (ResultSet resultSet = statement.executeQuery("select \"status,01\" from root.ln.d1")) {
+ int cnt = 0;
+ while (resultSet.next()) {
+ cnt++;
+ }
+ Assert.assertEquals(2, cnt);
+ }
+
statement.execute("DELETE FROM root.ln.d1.\"status,01\" WHERE time <= 400");
try (ResultSet resultSet = statement.executeQuery("select \"status,01\" from root.ln.d1")) {
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBEngineTimeGeneratorIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBEngineTimeGeneratorIT.java
index 243a5b9..3f1af95 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBEngineTimeGeneratorIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBEngineTimeGeneratorIT.java
@@ -202,7 +202,7 @@
RawDataQueryPlan queryPlan = new RawDataQueryPlan();
List<PartialPath> paths = new ArrayList<>();
paths.add(pd0s0);
- queryPlan.setDeduplicatedPaths(paths);
+ queryPlan.setDeduplicatedPathsAndUpdate(paths);
queryPlan.setExpression(singleSeriesExpression);
ServerTimeGenerator timeGenerator = new ServerTimeGenerator(TEST_QUERY_CONTEXT, queryPlan);
@@ -230,7 +230,7 @@
RawDataQueryPlan queryPlan = new RawDataQueryPlan();
List<PartialPath> paths = new ArrayList<>();
paths.add(pd1s0);
- queryPlan.setDeduplicatedPaths(paths);
+ queryPlan.setDeduplicatedPathsAndUpdate(paths);
queryPlan.setExpression(singleSeriesExpression);
ServerTimeGenerator timeGenerator = new ServerTimeGenerator(TEST_QUERY_CONTEXT, queryPlan);
@@ -268,7 +268,7 @@
List<PartialPath> paths = new ArrayList<>();
paths.add(pd0s0);
paths.add(pd0s2);
- queryPlan.setDeduplicatedPaths(paths);
+ queryPlan.setDeduplicatedPathsAndUpdate(paths);
queryPlan.setExpression(andExpression);
ServerTimeGenerator timeGenerator = new ServerTimeGenerator(TEST_QUERY_CONTEXT, queryPlan);
int cnt = 0;
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFlushQueryMergeIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFlushQueryMergeIT.java
index a936774..94a46ac 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFlushQueryMergeIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFlushQueryMergeIT.java
@@ -203,7 +203,9 @@
new StorageGroupNotSetException("root.notExistGroup1,root.notExistGroup2");
SQLException sqlException =
new SQLException(
- TSStatusCode.METADATA_ERROR.getStatusCode() + ": " + tmpsgnse.getMessage());
+ TSStatusCode.STORAGE_GROUP_NOT_EXIST.getStatusCode()
+ + ": "
+ + tmpsgnse.getMessage());
assertEquals(sqlException.getMessage(), sqe.getMessage());
}
} catch (Exception e) {
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java
index 540bf53..2c69a52 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java
@@ -210,7 +210,7 @@
TEST_QUERY_CONTEXT = new QueryContext(TEST_QUERY_JOB_ID);
RawDataQueryPlan queryPlan = new RawDataQueryPlan();
queryPlan.setDeduplicatedDataTypes(dataTypes);
- queryPlan.setDeduplicatedPaths(pathList);
+ queryPlan.setDeduplicatedPathsAndUpdate(pathList);
QueryDataSet queryDataSet = queryRouter.rawDataQuery(queryPlan, TEST_QUERY_CONTEXT);
int cnt = 0;
@@ -246,7 +246,7 @@
RawDataQueryPlan queryPlan = new RawDataQueryPlan();
queryPlan.setDeduplicatedDataTypes(dataTypes);
- queryPlan.setDeduplicatedPaths(pathList);
+ queryPlan.setDeduplicatedPathsAndUpdate(pathList);
queryPlan.setExpression(globalTimeExpression);
QueryDataSet queryDataSet = queryRouter.rawDataQuery(queryPlan, TEST_QUERY_CONTEXT);
@@ -303,7 +303,7 @@
RawDataQueryPlan queryPlan = new RawDataQueryPlan();
queryPlan.setDeduplicatedDataTypes(dataTypes);
- queryPlan.setDeduplicatedPaths(pathList);
+ queryPlan.setDeduplicatedPathsAndUpdate(pathList);
queryPlan.setExpression(singleSeriesExpression);
QueryDataSet queryDataSet = queryRouter.rawDataQuery(queryPlan, TEST_QUERY_CONTEXT);
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java
index 1710979..27b8abc 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java
@@ -319,7 +319,7 @@
RawDataQueryPlan queryPlan = new RawDataQueryPlan();
queryPlan.setDeduplicatedDataTypes(dataTypes);
- queryPlan.setDeduplicatedPaths(pathList);
+ queryPlan.setDeduplicatedPathsAndUpdate(pathList);
QueryDataSet queryDataSet = queryRouter.rawDataQuery(queryPlan, TEST_QUERY_CONTEXT);
int cnt = 0;
@@ -351,7 +351,7 @@
RawDataQueryPlan queryPlan = new RawDataQueryPlan();
queryPlan.setDeduplicatedDataTypes(dataTypes);
- queryPlan.setDeduplicatedPaths(pathList);
+ queryPlan.setDeduplicatedPathsAndUpdate(pathList);
queryPlan.setExpression(singleSeriesExpression);
QueryDataSet queryDataSet = queryRouter.rawDataQuery(queryPlan, TEST_QUERY_CONTEXT);
@@ -379,7 +379,7 @@
RawDataQueryPlan queryPlan = new RawDataQueryPlan();
queryPlan.setDeduplicatedDataTypes(dataTypes);
- queryPlan.setDeduplicatedPaths(Collections.singletonList(path));
+ queryPlan.setDeduplicatedPathsAndUpdate(Collections.singletonList(path));
queryPlan.setExpression(expression);
QueryDataSet queryDataSet = queryRouter.rawDataQuery(queryPlan, TEST_QUERY_CONTEXT);
@@ -407,7 +407,7 @@
List<PartialPath> pathList = new ArrayList<>();
pathList.add(path1);
pathList.add(path2);
- queryPlan.setDeduplicatedPaths(pathList);
+ queryPlan.setDeduplicatedPathsAndUpdate(pathList);
List<TSDataType> dataTypes = new ArrayList<>();
dataTypes.add(TSDataType.INT32);
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSimpleQueryIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSimpleQueryIT.java
index 533ebec..02cb24a 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSimpleQueryIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSimpleQueryIT.java
@@ -51,14 +51,18 @@
public class IoTDBSimpleQueryIT {
+ boolean autoCreateSchemaEnabled;
+
@Before
public void setUp() {
EnvironmentUtils.envSetUp();
+ autoCreateSchemaEnabled = IoTDBDescriptor.getInstance().getConfig().isAutoCreateSchemaEnabled();
}
@After
public void tearDown() throws Exception {
EnvironmentUtils.cleanEnv();
+ IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(autoCreateSchemaEnabled);
}
@Test
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagIT.java
index ad6a17f..3364318 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagIT.java
@@ -1136,26 +1136,30 @@
}
boolean hasResult = statement.execute("select s1, temperature from root.turbine.d1");
assertTrue(hasResult);
- ResultSet resultSet = statement.getResultSet();
- int count = 0;
- try {
- while (resultSet.next()) {
- String ans =
- resultSet.getString("Time")
- + ","
- + resultSet.getString("root.turbine.d1.s1")
- + ","
- + resultSet.getString("root.turbine.d1.s1");
- assertTrue(ret.contains(ans));
- count++;
- }
- } finally {
- resultSet.close();
- }
- assertEquals(ret.size(), count);
+ // FIXME should use the same reader for measurement and its alias
+
+ // ResultSet resultSet = statement.getResultSet();
+ // int count = 0;
+ // try {
+ // while (resultSet.next()) {
+ // String ans =
+ // resultSet.getString("Time")
+ // + ","
+ // + resultSet.getString("root.turbine.d1.s1")
+ // + ","
+ // + resultSet.getString("root.turbine.d1.s1");
+ // assertTrue(ret.contains(ans));
+ // count++;
+ // }
+ // } finally {
+ // resultSet.close();
+ // }
+ // assertEquals(ret.size(), count);
} catch (Exception e) {
e.printStackTrace();
- fail();
+ assertEquals(
+ "411: Error occurred in query process: Query for measurement and its alias at the same time!",
+ e.getMessage());
}
}
}
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTtlIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTtlIT.java
index 62f458e..252a5eb 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTtlIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTtlIT.java
@@ -75,7 +75,7 @@
try {
statement.execute("SET TTL TO root.TTL_SG1.s1 1000");
} catch (SQLException e) {
- assertEquals(TSStatusCode.METADATA_ERROR.getStatusCode(), e.getErrorCode());
+ assertEquals(TSStatusCode.STORAGE_GROUP_NOT_EXIST.getStatusCode(), e.getErrorCode());
}
long now = System.currentTimeMillis();
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
index 624f441..ab13c55 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
@@ -21,12 +21,22 @@
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.mnode.MNode;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.db.qp.physical.crud.SetDeviceTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
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.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.junit.After;
import org.junit.Assert;
@@ -44,6 +54,7 @@
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@@ -64,7 +75,6 @@
@Test
public void testAddPathAndExist() throws IllegalPathException {
-
MManager manager = IoTDB.metaManager;
assertTrue(manager.isPathExist(new PartialPath("root")));
@@ -114,19 +124,19 @@
TSDataType.INT32,
TSEncoding.RLE,
TSFileDescriptor.getInstance().getConfig().getCompressor(),
- Collections.EMPTY_MAP);
+ Collections.emptyMap());
manager.createTimeseries(
new PartialPath("root.laptop.d1.\"1.2.3\""),
TSDataType.INT32,
TSEncoding.RLE,
TSFileDescriptor.getInstance().getConfig().getCompressor(),
- Collections.EMPTY_MAP);
+ Collections.emptyMap());
manager.createTimeseries(
new PartialPath("root.1.2.3"),
TSDataType.INT32,
TSEncoding.RLE,
TSFileDescriptor.getInstance().getConfig().getCompressor(),
- Collections.EMPTY_MAP);
+ Collections.emptyMap());
assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.s1")));
assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.1_2")));
@@ -230,6 +240,126 @@
assertFalse(manager.isPathExist(new PartialPath("root.1")));
}
+ @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
+ @Test
+ public void testCreateAlignedTimeseries() throws IllegalPathException {
+ MManager manager = IoTDB.metaManager;
+ try {
+ manager.setStorageGroup(new PartialPath("root.laptop"));
+ } catch (MetadataException e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+
+ try {
+ manager.createTimeseries(
+ new PartialPath("root.laptop.d1.s0"),
+ TSDataType.valueOf("INT32"),
+ TSEncoding.valueOf("RLE"),
+ compressionType,
+ Collections.emptyMap());
+ manager.createAlignedTimeSeries(
+ new PartialPath("root.laptop.d1"),
+ Arrays.asList("s1", "s2", "s3"),
+ Arrays.asList(
+ TSDataType.valueOf("INT32"),
+ TSDataType.valueOf("FLOAT"),
+ TSDataType.valueOf("INT32")),
+ Arrays.asList(
+ TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE")),
+ compressionType);
+ } catch (MetadataException e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+
+ assertTrue(manager.isPathExist(new PartialPath("root.laptop")));
+ assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1")));
+ assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.s0")));
+ assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.s1")));
+ assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.s2")));
+ assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.s3")));
+ try {
+ assertEquals(
+ 1,
+ manager
+ .getStorageGroupNodeByStorageGroupPath(new PartialPath("root.laptop"))
+ .getAlignedTimeseriesIndex());
+ } catch (MetadataException e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+
+ try {
+ manager.deleteTimeseries(new PartialPath("root.laptop.d1.s2"));
+ } catch (MetadataException e) {
+ assertEquals(
+ "Not support deleting part of aligned timeseies! (Path: root.laptop.d1.s2)",
+ e.getMessage());
+ }
+
+ try {
+ manager.deleteTimeseries(new PartialPath("root.laptop.d1.(s2, s3)"));
+ } catch (MetadataException e) {
+ assertEquals(
+ "Not support deleting part of aligned timeseies! (Path: root.laptop.d1.(s2, s3))",
+ e.getMessage());
+ }
+
+ try {
+ manager.deleteTimeseries(new PartialPath("root.laptop.d1.(s1,s2,s3)"));
+ } catch (MetadataException e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+
+ assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1")));
+ assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.s0")));
+ assertFalse(manager.isPathExist(new PartialPath("root.laptop.d1.s1")));
+ assertFalse(manager.isPathExist(new PartialPath("root.laptop.d1.s2")));
+ assertFalse(manager.isPathExist(new PartialPath("root.laptop.d1.s3")));
+
+ try {
+ manager.deleteTimeseries(new PartialPath("root.laptop.d1.s0"));
+ } catch (MetadataException e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ assertFalse(manager.isPathExist(new PartialPath("root.laptop.d1")));
+ assertFalse(manager.isPathExist(new PartialPath("root.laptop.d1.s0")));
+
+ try {
+ manager.createAlignedTimeSeries(
+ new PartialPath("root.laptop.d1"),
+ Arrays.asList("s0", "s2", "s4"),
+ Arrays.asList(
+ TSDataType.valueOf("INT32"),
+ TSDataType.valueOf("FLOAT"),
+ TSDataType.valueOf("INT32")),
+ Arrays.asList(
+ TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE")),
+ compressionType);
+ } catch (MetadataException e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+
+ assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1")));
+ assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.s0")));
+ assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.s2")));
+ assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.s4")));
+ try {
+ assertEquals(
+ 2,
+ manager
+ .getStorageGroupNodeByStorageGroupPath(new PartialPath("root.laptop"))
+ .getAlignedTimeseriesIndex());
+ } catch (MetadataException e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
@Test
public void testGetAllTimeseriesCount() {
MManager manager = IoTDB.metaManager;
@@ -736,4 +866,401 @@
fail(e.getMessage());
}
}
+
+ @Test
+ public void testTemplate() throws MetadataException {
+ CreateTemplatePlan plan = getCreateTemplatePlan();
+
+ MManager manager = IoTDB.metaManager;
+ manager.createDeviceTemplate(plan);
+
+ // set device template
+ SetDeviceTemplatePlan setDeviceTemplatePlan =
+ new SetDeviceTemplatePlan("template1", "root.sg1.d1");
+
+ manager.setDeviceTemplate(setDeviceTemplatePlan);
+
+ MNode node = manager.getDeviceNode(new PartialPath("root.sg1.d1"));
+ node.setUseTemplate(true);
+
+ MeasurementSchema s11 =
+ new MeasurementSchema("s11", TSDataType.INT64, TSEncoding.RLE, CompressionType.SNAPPY);
+ assertNotNull(node.getDeviceTemplate());
+ assertEquals(node.getDeviceTemplate().getSchemaMap().get("s11"), s11);
+
+ Set<IMeasurementSchema> allSchema =
+ new HashSet<>(node.getDeviceTemplate().getSchemaMap().values());
+ for (IMeasurementSchema schema :
+ manager.getAllMeasurementByDevicePath(new PartialPath("root.sg1.d1"))) {
+ allSchema.remove(schema);
+ }
+
+ assertTrue(allSchema.isEmpty());
+ }
+
+ private CreateTemplatePlan getCreateTemplatePlan() {
+ List<List<String>> measurementList = new ArrayList<>();
+ measurementList.add(Collections.singletonList("s11"));
+ List<String> measurements = new ArrayList<>();
+ for (int i = 0; i < 10; i++) {
+ measurements.add("s" + i);
+ }
+ measurementList.add(measurements);
+
+ List<List<TSDataType>> dataTypeList = new ArrayList<>();
+ dataTypeList.add(Collections.singletonList(TSDataType.INT64));
+ List<TSDataType> dataTypes = new ArrayList<>();
+ for (int i = 0; i < 10; i++) {
+ dataTypes.add(TSDataType.INT64);
+ }
+ dataTypeList.add(dataTypes);
+
+ List<List<TSEncoding>> encodingList = new ArrayList<>();
+ encodingList.add(Collections.singletonList(TSEncoding.RLE));
+ List<TSEncoding> encodings = new ArrayList<>();
+ for (int i = 0; i < 10; i++) {
+ encodings.add(TSEncoding.RLE);
+ }
+ encodingList.add(encodings);
+
+ List<CompressionType> compressionTypes = new ArrayList<>();
+ for (int i = 0; i < 11; i++) {
+ compressionTypes.add(CompressionType.SNAPPY);
+ }
+
+ return new CreateTemplatePlan(
+ "template1", measurementList, dataTypeList, encodingList, compressionTypes);
+ }
+
+ @Test
+ public void testTemplateCompatibility() throws MetadataException {
+ List<List<String>> measurementList = new ArrayList<>();
+ measurementList.add(Collections.singletonList("s11"));
+ List<String> measurements = new ArrayList<>();
+ for (int i = 0; i < 10; i++) {
+ measurements.add("s" + i);
+ }
+ measurementList.add(measurements);
+
+ List<List<TSDataType>> dataTypeList = new ArrayList<>();
+ dataTypeList.add(Collections.singletonList(TSDataType.INT64));
+ List<TSDataType> dataTypes = new ArrayList<>();
+ for (int i = 0; i < 10; i++) {
+ dataTypes.add(TSDataType.INT64);
+ }
+ dataTypeList.add(dataTypes);
+
+ List<List<TSEncoding>> encodingList = new ArrayList<>();
+ encodingList.add(Collections.singletonList(TSEncoding.RLE));
+ List<TSEncoding> encodings = new ArrayList<>();
+ for (int i = 0; i < 10; i++) {
+ encodings.add(TSEncoding.RLE);
+ }
+ encodingList.add(encodings);
+
+ List<CompressionType> compressionTypes = new ArrayList<>();
+ for (int i = 0; i < 11; i++) {
+ compressionTypes.add(CompressionType.SNAPPY);
+ }
+
+ CreateTemplatePlan plan1 =
+ new CreateTemplatePlan(
+ "template1",
+ new ArrayList<>(measurementList),
+ new ArrayList<>(dataTypeList),
+ new ArrayList<>(encodingList),
+ new ArrayList<>(compressionTypes));
+
+ measurementList.add(Collections.singletonList("s12"));
+ dataTypeList.add(Collections.singletonList(TSDataType.INT64));
+ encodingList.add(Collections.singletonList(TSEncoding.RLE));
+ compressionTypes.add(CompressionType.SNAPPY);
+
+ CreateTemplatePlan plan2 =
+ new CreateTemplatePlan(
+ "template2",
+ new ArrayList<>(measurementList),
+ new ArrayList<>(dataTypeList),
+ new ArrayList<>(encodingList),
+ new ArrayList<>(compressionTypes));
+
+ MManager manager = IoTDB.metaManager;
+
+ assertTrue(manager.isTemplateCompatible(new Template(plan1), new Template(plan2)));
+ assertFalse(manager.isTemplateCompatible(new Template(plan2), new Template(plan1)));
+
+ System.out.println(measurementList);
+ measurementList.get(1).add("s13");
+ dataTypeList.get(1).add(TSDataType.INT64);
+ encodingList.get(1).add(TSEncoding.RLE);
+
+ CreateTemplatePlan plan3 =
+ new CreateTemplatePlan(
+ "template3",
+ new ArrayList<>(measurementList),
+ new ArrayList<>(dataTypeList),
+ new ArrayList<>(encodingList),
+ new ArrayList<>(compressionTypes));
+
+ assertTrue(manager.isTemplateCompatible(new Template(plan1), new Template(plan3)));
+
+ List<String> vectorList = new ArrayList<>(measurementList.get(1));
+ vectorList.remove(0);
+ List<TSDataType> vectorDataTypesList = new ArrayList<>(dataTypeList.get(1));
+ vectorDataTypesList.remove(0);
+ List<TSEncoding> vectorEncodingsList = new ArrayList<>(encodingList.get(1));
+ vectorEncodingsList.remove(0);
+
+ measurementList.set(1, vectorList);
+ dataTypeList.set(1, vectorDataTypesList);
+ encodingList.set(1, vectorEncodingsList);
+
+ CreateTemplatePlan plan4 =
+ new CreateTemplatePlan(
+ "template4",
+ new ArrayList<>(measurementList),
+ new ArrayList<>(dataTypeList),
+ new ArrayList<>(encodingList),
+ new ArrayList<>(compressionTypes));
+
+ assertFalse(manager.isTemplateCompatible(new Template(plan1), new Template(plan4)));
+
+ // test manager
+ manager.createDeviceTemplate(plan1);
+ manager.createDeviceTemplate(plan2);
+ manager.createDeviceTemplate(plan4);
+
+ manager.setDeviceTemplate(new SetDeviceTemplatePlan("template1", "root.sg1.d1"));
+ try {
+ manager.setDeviceTemplate(new SetDeviceTemplatePlan("template4", "root.sg1.d1.d2"));
+ fail("These two templates are incompatible");
+ } catch (MetadataException e) {
+ assertEquals("Incompatible template", e.getMessage());
+ }
+
+ manager.setDeviceTemplate(new SetDeviceTemplatePlan("template2", "root.sg1.d1.d2"));
+ }
+
+ @Test
+ public void testTemplateAndTimeSeriesCompatibility() throws MetadataException {
+ CreateTemplatePlan plan = getCreateTemplatePlan();
+ MManager manager = IoTDB.metaManager;
+ manager.createDeviceTemplate(plan);
+
+ // set device template
+ SetDeviceTemplatePlan setDeviceTemplatePlan =
+ new SetDeviceTemplatePlan("template1", "root.sg1.d1");
+
+ manager.setDeviceTemplate(setDeviceTemplatePlan);
+
+ CreateTimeSeriesPlan createTimeSeriesPlan =
+ new CreateTimeSeriesPlan(
+ new PartialPath("root.sg1.d1.s20"),
+ TSDataType.INT32,
+ TSEncoding.PLAIN,
+ CompressionType.GZIP,
+ null,
+ null,
+ null,
+ null);
+
+ manager.createTimeseries(createTimeSeriesPlan);
+
+ CreateTimeSeriesPlan createTimeSeriesPlan2 =
+ new CreateTimeSeriesPlan(
+ new PartialPath("root.sg1.d1.s1"),
+ TSDataType.INT32,
+ TSEncoding.PLAIN,
+ CompressionType.GZIP,
+ null,
+ null,
+ null,
+ null);
+
+ try {
+ manager.createTimeseries(createTimeSeriesPlan2);
+ fail();
+ } catch (Exception e) {
+ assertEquals(
+ "Path [root.sg1.d1.s1 ( which is incompatible with template )] already exist",
+ e.getMessage());
+ }
+ }
+
+ @Test
+ public void testShowTimeseries() {
+ MManager manager = IoTDB.metaManager;
+ try {
+ manager.createTimeseries(
+ new PartialPath("root.laptop.d1.s0"),
+ TSDataType.valueOf("INT32"),
+ TSEncoding.valueOf("RLE"),
+ compressionType,
+ Collections.emptyMap());
+ manager.createAlignedTimeSeries(
+ new PartialPath("root.laptop.d1"),
+ Arrays.asList("s1", "s2", "s3"),
+ Arrays.asList(
+ TSDataType.valueOf("INT32"),
+ TSDataType.valueOf("FLOAT"),
+ TSDataType.valueOf("INT32")),
+ Arrays.asList(
+ TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE")),
+ compressionType);
+
+ // show timeseries root.laptop.d1.s0
+ ShowTimeSeriesPlan showTimeSeriesPlan =
+ new ShowTimeSeriesPlan(
+ new PartialPath("root.laptop.d1.s0"), false, null, null, 0, 0, false);
+ List<ShowTimeSeriesResult> result =
+ manager.showTimeseries(showTimeSeriesPlan, new QueryContext());
+ assertEquals(1, result.size());
+ assertEquals("root.laptop.d1.s0", result.get(0).getName());
+
+ // show timeseries root.laptop.d1.s1
+ showTimeSeriesPlan =
+ new ShowTimeSeriesPlan(
+ new PartialPath("root.laptop.d1.s1"), false, null, null, 0, 0, false);
+ result = manager.showTimeseries(showTimeSeriesPlan, new QueryContext());
+ assertEquals(1, result.size());
+ assertEquals("root.laptop.d1.s1", result.get(0).getName());
+
+ // show timeseries root.laptop.d1.(s1,s2,s3)
+ showTimeSeriesPlan =
+ new ShowTimeSeriesPlan(
+ new PartialPath("root.laptop.d1.(s1,s2,s3)"), false, null, null, 0, 0, false);
+ result = manager.showTimeseries(showTimeSeriesPlan, new QueryContext());
+ assertEquals(3, result.size());
+ for (int i = 0; i < result.size(); i++) {
+ assertEquals("root.laptop.d1.s" + (i + 1), result.get(i).getName());
+ }
+
+ // show timeseries
+ showTimeSeriesPlan =
+ new ShowTimeSeriesPlan(new PartialPath("root"), false, null, null, 0, 0, false);
+ result = manager.showTimeseries(showTimeSeriesPlan, new QueryContext());
+ assertEquals(4, result.size());
+ } catch (MetadataException e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+
+ // show timeseries root.laptop.d1.(s0,s1)
+ try {
+ ShowTimeSeriesPlan showTimeSeriesPlan =
+ new ShowTimeSeriesPlan(
+ new PartialPath("root.laptop.d1.(s0,s1)"), false, null, null, 0, 0, false);
+ List<ShowTimeSeriesResult> result =
+ manager.showTimeseries(showTimeSeriesPlan, new QueryContext());
+ } catch (MetadataException e) {
+ assertEquals(
+ "Cannot get node of children in different aligned timeseries (Path: (s0,s1))",
+ e.getMessage());
+ }
+ }
+
+ @Test
+ public void testShowTimeseriesWithTemplate() {
+ List<List<String>> measurementList = new ArrayList<>();
+ measurementList.add(Collections.singletonList("s0"));
+ List<String> measurements = new ArrayList<>();
+ for (int i = 1; i <= 3; i++) {
+ measurements.add("s" + i);
+ }
+ measurementList.add(measurements);
+
+ List<List<TSDataType>> dataTypeList = new ArrayList<>();
+ dataTypeList.add(Collections.singletonList(TSDataType.INT32));
+ List<TSDataType> dataTypes = new ArrayList<>();
+ dataTypes.add(TSDataType.INT32);
+ dataTypes.add(TSDataType.FLOAT);
+ dataTypes.add(TSDataType.INT32);
+ dataTypeList.add(dataTypes);
+
+ List<List<TSEncoding>> encodingList = new ArrayList<>();
+ encodingList.add(Collections.singletonList(TSEncoding.RLE));
+ List<TSEncoding> encodings = new ArrayList<>();
+ for (int i = 1; i <= 3; i++) {
+ encodings.add(TSEncoding.RLE);
+ }
+ encodingList.add(encodings);
+
+ List<CompressionType> compressionTypes = new ArrayList<>();
+ for (int i = 0; i < 2; i++) {
+ compressionTypes.add(compressionType);
+ }
+
+ CreateTemplatePlan plan =
+ new CreateTemplatePlan(
+ "template1", measurementList, dataTypeList, encodingList, compressionTypes);
+ MManager manager = IoTDB.metaManager;
+ try {
+ manager.createDeviceTemplate(plan);
+
+ // set device template
+ SetDeviceTemplatePlan setDeviceTemplatePlan =
+ new SetDeviceTemplatePlan("template1", "root.laptop.d1");
+ manager.setDeviceTemplate(setDeviceTemplatePlan);
+ manager.getDeviceNode(new PartialPath("root.laptop.d1")).setUseTemplate(true);
+
+ // show timeseries root.laptop.d1.s0
+ ShowTimeSeriesPlan showTimeSeriesPlan =
+ new ShowTimeSeriesPlan(
+ new PartialPath("root.laptop.d1.s0"), false, null, null, 0, 0, false);
+ List<ShowTimeSeriesResult> result =
+ manager.showTimeseries(showTimeSeriesPlan, new QueryContext());
+ assertEquals(1, result.size());
+ assertEquals("root.laptop.d1.s0", result.get(0).getName());
+
+ // show timeseries root.laptop.d1.s1
+ showTimeSeriesPlan =
+ new ShowTimeSeriesPlan(
+ new PartialPath("root.laptop.d1.s1"), false, null, null, 0, 0, false);
+ result = manager.showTimeseries(showTimeSeriesPlan, new QueryContext());
+ assertEquals(1, result.size());
+ assertEquals("root.laptop.d1.s1", result.get(0).getName());
+
+ // show timeseries root.laptop.d1.(s1,s2,s3)
+ showTimeSeriesPlan =
+ new ShowTimeSeriesPlan(
+ new PartialPath("root.laptop.d1.(s1,s2,s3)"), false, null, null, 0, 0, false);
+ result = manager.showTimeseries(showTimeSeriesPlan, new QueryContext());
+ assertEquals(3, result.size());
+ for (int i = 0; i < result.size(); i++) {
+ assertEquals("root.laptop.d1.s" + (i + 1), result.get(i).getName());
+ }
+
+ // show timeseries root.laptop.d1.(s1,s2,s3)
+ showTimeSeriesPlan =
+ new ShowTimeSeriesPlan(new PartialPath("root"), false, null, null, 0, 0, false);
+ result = manager.showTimeseries(showTimeSeriesPlan, new QueryContext());
+ assertEquals(4, result.size());
+ Set<String> set = new HashSet<>();
+ for (int i = 0; i < result.size(); i++) {
+ set.add("root.laptop.d1.s" + i);
+ }
+
+ for (int i = 0; i < result.size(); i++) {
+ set.remove(result.get(i).getName());
+ }
+
+ assertTrue(set.isEmpty());
+ } catch (MetadataException e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+
+ // show timeseries root.laptop.d1.(s0,s1)
+ try {
+ ShowTimeSeriesPlan showTimeSeriesPlan =
+ new ShowTimeSeriesPlan(
+ new PartialPath("root.laptop.d1.(s0,s1)"), false, null, null, 0, 0, false);
+ List<ShowTimeSeriesResult> result =
+ manager.showTimeseries(showTimeSeriesPlan, new QueryContext());
+ } catch (MetadataException e) {
+ assertEquals(
+ "Cannot get node of children in different aligned timeseries (Path: (s0,s1))",
+ e.getMessage());
+ }
+ }
}
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerImproveTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerImproveTest.java
index e7212f0..4b52b54 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerImproveTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerImproveTest.java
@@ -35,6 +35,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
@@ -136,12 +137,16 @@
}
private void doCacheTest(String deviceId, List<String> measurementList) throws MetadataException {
- MNode node = mManager.getDeviceNodeWithAutoCreate(new PartialPath(deviceId));
- for (String s : measurementList) {
- assertTrue(node.hasChild(s));
- MeasurementMNode measurementNode = (MeasurementMNode) node.getChild(s);
- TSDataType dataType = measurementNode.getSchema().getType();
- assertEquals(TSDataType.TEXT, dataType);
+ try {
+ MNode node = mManager.getDeviceNodeWithAutoCreate(new PartialPath(deviceId)).left;
+ for (String s : measurementList) {
+ assertTrue(node.hasChild(s));
+ MeasurementMNode measurementNode = (MeasurementMNode) node.getChild(s);
+ TSDataType dataType = measurementNode.getSchema().getType();
+ assertEquals(TSDataType.TEXT, dataType);
+ }
+ } catch (IOException e) {
+ throw new MetadataException(e);
}
}
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertRowPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertRowPlanTest.java
new file mode 100644
index 0000000..3fe2736
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertRowPlanTest.java
@@ -0,0 +1,246 @@
+/*
+ * 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;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.Planner;
+import org.apache.iotdb.db.qp.executor.PlanExecutor;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan.PhysicalPlanType;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.db.qp.physical.crud.SetDeviceTemplatePlan;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+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.common.RowRecord;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+public class InsertRowPlanTest {
+
+ private final Planner processor = new Planner();
+
+ @Before
+ public void before() {
+ EnvironmentUtils.envSetUp();
+ }
+
+ @After
+ public void clean() throws IOException, StorageEngineException {
+ IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(true);
+ EnvironmentUtils.cleanEnv();
+ }
+
+ @Test
+ public void testInsertRowPlan()
+ throws QueryProcessException, MetadataException, InterruptedException,
+ QueryFilterOptimizationException, StorageEngineException, IOException {
+ long time = 110L;
+ TSDataType[] dataTypes =
+ new TSDataType[] {
+ TSDataType.DOUBLE,
+ TSDataType.FLOAT,
+ TSDataType.INT64,
+ TSDataType.INT32,
+ TSDataType.BOOLEAN,
+ TSDataType.TEXT
+ };
+
+ String[] columns = new String[6];
+ columns[0] = 1.0 + "";
+ columns[1] = 2 + "";
+ columns[2] = 10000 + "";
+ columns[3] = 100 + "";
+ columns[4] = false + "";
+ columns[5] = "hh" + 0;
+
+ InsertRowPlan rowPlan =
+ new InsertRowPlan(
+ new PartialPath("root.isp.d1"),
+ time,
+ new String[] {"s1", "s2", "s3", "s4", "s5", "s6"},
+ dataTypes,
+ columns);
+
+ PlanExecutor executor = new PlanExecutor();
+ executor.insert(rowPlan);
+
+ QueryPlan queryPlan = (QueryPlan) processor.parseSQLToPhysicalPlan("select * from root.isp.d1");
+ QueryDataSet dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+ Assert.assertEquals(6, dataSet.getPaths().size());
+ while (dataSet.hasNext()) {
+ RowRecord record = dataSet.next();
+ Assert.assertEquals(6, record.getFields().size());
+ }
+ }
+
+ @Test
+ public void testInsertRowPlanWithAlignedTimeseries()
+ throws QueryProcessException, MetadataException, InterruptedException,
+ QueryFilterOptimizationException, StorageEngineException, IOException {
+ InsertRowPlan vectorRowPlan = getInsertVectorRowPlan();
+
+ PlanExecutor executor = new PlanExecutor();
+ executor.insert(vectorRowPlan);
+
+ Assert.assertEquals("[$#$0, $#$1, s6]", Arrays.toString(vectorRowPlan.getMeasurementMNodes()));
+
+ QueryPlan queryPlan = (QueryPlan) processor.parseSQLToPhysicalPlan("select * from root.isp.d1");
+ QueryDataSet dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+ Assert.assertEquals(3, dataSet.getPaths().size());
+ while (dataSet.hasNext()) {
+ RowRecord record = dataSet.next();
+ Assert.assertEquals(6, record.getFields().size());
+ }
+ }
+
+ @Test
+ public void testInsertRowPlanWithDeviceTemplate()
+ throws QueryProcessException, MetadataException, InterruptedException,
+ QueryFilterOptimizationException, StorageEngineException, IOException {
+ List<List<String>> measurementList = new ArrayList<>();
+ List<String> v1 = new ArrayList<>();
+ v1.add("s1");
+ v1.add("s2");
+ v1.add("s3");
+ measurementList.add(v1);
+ List<String> v2 = new ArrayList<>();
+ v2.add("s4");
+ v2.add("s5");
+ measurementList.add(v2);
+ measurementList.add(Collections.singletonList("s6"));
+
+ List<List<TSDataType>> dataTypesList = new ArrayList<>();
+ List<TSDataType> d1 = new ArrayList<>();
+ d1.add(TSDataType.DOUBLE);
+ d1.add(TSDataType.FLOAT);
+ d1.add(TSDataType.INT64);
+ dataTypesList.add(d1);
+ List<TSDataType> d2 = new ArrayList<>();
+ d2.add(TSDataType.INT32);
+ d2.add(TSDataType.BOOLEAN);
+ dataTypesList.add(d2);
+ dataTypesList.add(Collections.singletonList(TSDataType.TEXT));
+
+ List<List<TSEncoding>> encodingList = new ArrayList<>();
+ List<TSEncoding> e1 = new ArrayList<>();
+ e1.add(TSEncoding.PLAIN);
+ e1.add(TSEncoding.PLAIN);
+ e1.add(TSEncoding.PLAIN);
+ encodingList.add(e1);
+ List<TSEncoding> e2 = new ArrayList<>();
+ e2.add(TSEncoding.PLAIN);
+ e2.add(TSEncoding.PLAIN);
+ encodingList.add(e2);
+ encodingList.add(Collections.singletonList(TSEncoding.PLAIN));
+
+ List<CompressionType> compressionTypes = new ArrayList<>();
+ for (int i = 0; i < 3; i++) {
+ compressionTypes.add(CompressionType.SNAPPY);
+ }
+
+ CreateTemplatePlan plan =
+ new CreateTemplatePlan(
+ "template1", measurementList, dataTypesList, encodingList, compressionTypes);
+
+ IoTDB.metaManager.createDeviceTemplate(plan);
+ IoTDB.metaManager.setDeviceTemplate(new SetDeviceTemplatePlan("template1", "root.isp.d1"));
+
+ IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(false);
+
+ InsertRowPlan rowPlan = getInsertVectorRowPlan();
+
+ PlanExecutor executor = new PlanExecutor();
+ executor.insert(rowPlan);
+
+ QueryPlan queryPlan = (QueryPlan) processor.parseSQLToPhysicalPlan("select * from root.isp.d1");
+ QueryDataSet dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+ Assert.assertEquals(3, dataSet.getPaths().size());
+ while (dataSet.hasNext()) {
+ RowRecord record = dataSet.next();
+ Assert.assertEquals(6, record.getFields().size());
+ }
+ }
+
+ @Test
+ public void testInsertRowSerialization() throws IllegalPathException, QueryProcessException {
+ InsertRowPlan plan1 = getInsertVectorRowPlan();
+
+ PlanExecutor executor = new PlanExecutor();
+ executor.insert(plan1);
+
+ ByteBuffer byteBuffer = ByteBuffer.allocate(10000);
+ plan1.serialize(byteBuffer);
+ byteBuffer.flip();
+
+ Assert.assertEquals(PhysicalPlanType.INSERT.ordinal(), byteBuffer.get());
+
+ InsertRowPlan plan2 = new InsertRowPlan();
+ plan2.deserialize(byteBuffer);
+ Assert.assertEquals(plan1, plan2);
+ }
+
+ private InsertRowPlan getInsertVectorRowPlan() throws IllegalPathException {
+ long time = 110L;
+ TSDataType[] dataTypes =
+ new TSDataType[] {
+ TSDataType.DOUBLE,
+ TSDataType.FLOAT,
+ TSDataType.INT64,
+ TSDataType.INT32,
+ TSDataType.BOOLEAN,
+ TSDataType.TEXT
+ };
+
+ String[] columns = new String[6];
+ columns[0] = 1.0 + "";
+ columns[1] = 2 + "";
+ columns[2] = 10000 + "";
+ columns[3] = 100 + "";
+ columns[4] = false + "";
+ columns[5] = "hh" + 0;
+
+ return new InsertRowPlan(
+ new PartialPath("root.isp.d1"),
+ time,
+ new String[] {"(s1,s2,s3)", "(s4,s5)", "s6"},
+ dataTypes,
+ columns);
+ }
+}
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertTabletPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertTabletPlanTest.java
index 52b5b64..17e6ab1 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertTabletPlanTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertTabletPlanTest.java
@@ -18,20 +18,29 @@
*/
package org.apache.iotdb.db.qp.physical;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.exception.metadata.MetadataException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.metadata.PartialPath;
import org.apache.iotdb.db.qp.Planner;
import org.apache.iotdb.db.qp.executor.PlanExecutor;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan.PhysicalPlanType;
+import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
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.SetDeviceTemplatePlan;
+import org.apache.iotdb.db.service.IoTDB;
import org.apache.iotdb.db.utils.EnvironmentUtils;
import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+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.common.RowRecord;
import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.BitMap;
import org.junit.After;
import org.junit.Assert;
@@ -39,7 +48,10 @@
import org.junit.Test;
import java.io.IOException;
+import java.nio.ByteBuffer;
import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
import java.util.List;
public class InsertTabletPlanTest {
@@ -53,6 +65,7 @@
@After
public void clean() throws IOException, StorageEngineException {
+ IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(true);
EnvironmentUtils.cleanEnv();
}
@@ -106,4 +119,255 @@
Assert.assertEquals(6, record.getFields().size());
}
}
+
+ @Test
+ public void testInsertTabletPlanWithAlignedTimeseries()
+ throws QueryProcessException, MetadataException, InterruptedException,
+ QueryFilterOptimizationException, StorageEngineException, IOException {
+ InsertTabletPlan tabletPlan = getInsertTabletPlan();
+
+ PlanExecutor executor = new PlanExecutor();
+ executor.insertTablet(tabletPlan);
+
+ Assert.assertEquals("[$#$0, $#$1, s6]", Arrays.toString(tabletPlan.getMeasurementMNodes()));
+
+ QueryPlan queryPlan = (QueryPlan) processor.parseSQLToPhysicalPlan("select * from root.isp.d1");
+ QueryDataSet dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+ Assert.assertEquals(3, dataSet.getPaths().size());
+ while (dataSet.hasNext()) {
+ RowRecord record = dataSet.next();
+ System.out.println(record);
+ Assert.assertEquals(6, record.getFields().size());
+ }
+ }
+
+ @Test
+ public void testInsertNullableTabletPlanWithAlignedTimeseries()
+ throws QueryProcessException, MetadataException, InterruptedException,
+ QueryFilterOptimizationException, StorageEngineException, IOException {
+ InsertTabletPlan tabletPlan = getInsertTabletPlan();
+ tabletPlan.setBitMaps(new BitMap[6]);
+ BitMap[] bitMaps = tabletPlan.getBitMaps();
+ for (int i = 0; i < 4; i++) {
+ if (bitMaps[i] == null) {
+ bitMaps[i] = new BitMap(4);
+ }
+ bitMaps[i].mark(i);
+ }
+ PlanExecutor executor = new PlanExecutor();
+ executor.insertTablet(tabletPlan);
+
+ Assert.assertEquals("[$#$0, $#$1, s6]", Arrays.toString(tabletPlan.getMeasurementMNodes()));
+ System.out.println(Arrays.toString(tabletPlan.getMeasurementMNodes()));
+
+ QueryPlan queryPlan = (QueryPlan) processor.parseSQLToPhysicalPlan("select * from root.isp.d1");
+ QueryDataSet dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+ Assert.assertEquals(3, dataSet.getPaths().size());
+ while (dataSet.hasNext()) {
+ RowRecord record = dataSet.next();
+ System.out.println(record);
+ Assert.assertEquals(6, record.getFields().size());
+ }
+ }
+
+ @Test
+ public void testInsertTabletPlanWithDeviceTemplate()
+ throws QueryProcessException, MetadataException, InterruptedException,
+ QueryFilterOptimizationException, StorageEngineException, IOException {
+ CreateTemplatePlan plan = getCreateTemplatePlan();
+
+ IoTDB.metaManager.createDeviceTemplate(plan);
+ IoTDB.metaManager.setDeviceTemplate(new SetDeviceTemplatePlan("template1", "root.isp"));
+
+ InsertTabletPlan tabletPlan = getInsertTabletPlan();
+
+ PlanExecutor executor = new PlanExecutor();
+
+ // nothing can be found when we not insert data
+ QueryPlan queryPlan = (QueryPlan) processor.parseSQLToPhysicalPlan("select * from root.isp");
+ QueryDataSet dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+ Assert.assertEquals(0, dataSet.getPaths().size());
+
+ executor.insertTablet(tabletPlan);
+
+ queryPlan = (QueryPlan) processor.parseSQLToPhysicalPlan("select * from root.isp");
+ dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+ Assert.assertEquals(3, dataSet.getPaths().size());
+ while (dataSet.hasNext()) {
+ RowRecord record = dataSet.next();
+ Assert.assertEquals(6, record.getFields().size());
+ }
+ }
+
+ private CreateTemplatePlan getCreateTemplatePlan() {
+ List<List<String>> measurementList = new ArrayList<>();
+ List<String> v1 = new ArrayList<>();
+ v1.add("s1");
+ v1.add("s2");
+ v1.add("s3");
+ measurementList.add(v1);
+ List<String> v2 = new ArrayList<>();
+ v2.add("s4");
+ v2.add("s5");
+ measurementList.add(v2);
+ measurementList.add(Collections.singletonList("s6"));
+
+ List<List<TSDataType>> dataTypesList = new ArrayList<>();
+ List<TSDataType> d1 = new ArrayList<>();
+ d1.add(TSDataType.DOUBLE);
+ d1.add(TSDataType.FLOAT);
+ d1.add(TSDataType.INT64);
+ dataTypesList.add(d1);
+ List<TSDataType> d2 = new ArrayList<>();
+ d2.add(TSDataType.INT32);
+ d2.add(TSDataType.BOOLEAN);
+ dataTypesList.add(d2);
+ dataTypesList.add(Collections.singletonList(TSDataType.TEXT));
+
+ List<List<TSEncoding>> encodingList = new ArrayList<>();
+ List<TSEncoding> e1 = new ArrayList<>();
+ e1.add(TSEncoding.PLAIN);
+ e1.add(TSEncoding.PLAIN);
+ e1.add(TSEncoding.PLAIN);
+ encodingList.add(e1);
+ List<TSEncoding> e2 = new ArrayList<>();
+ e2.add(TSEncoding.PLAIN);
+ e2.add(TSEncoding.PLAIN);
+ encodingList.add(e2);
+ encodingList.add(Collections.singletonList(TSEncoding.PLAIN));
+
+ List<CompressionType> compressionTypes = new ArrayList<>();
+ for (int i = 0; i < 3; i++) {
+ compressionTypes.add(CompressionType.SNAPPY);
+ }
+
+ return new CreateTemplatePlan(
+ "template1", measurementList, dataTypesList, encodingList, compressionTypes);
+ }
+
+ @Test
+ public void testInsertTabletPlanWithDeviceTemplateAndAutoCreateSchema()
+ throws QueryProcessException, MetadataException, InterruptedException,
+ QueryFilterOptimizationException, StorageEngineException, IOException {
+ CreateTemplatePlan plan = getCreateTemplatePlan();
+
+ IoTDB.metaManager.createDeviceTemplate(plan);
+ IoTDB.metaManager.setDeviceTemplate(new SetDeviceTemplatePlan("template1", "root.isp"));
+ InsertTabletPlan tabletPlan = getInsertTabletPlan();
+
+ PlanExecutor executor = new PlanExecutor();
+ executor.insertTablet(tabletPlan);
+
+ QueryPlan queryPlan = (QueryPlan) processor.parseSQLToPhysicalPlan("select * from root.isp.d1");
+ QueryDataSet dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+ Assert.assertEquals(3, dataSet.getPaths().size());
+ while (dataSet.hasNext()) {
+ RowRecord record = dataSet.next();
+ Assert.assertEquals(6, record.getFields().size());
+ }
+
+ // test recover
+ EnvironmentUtils.stopDaemon();
+ IoTDB.metaManager.clear();
+ // wait for close
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ Thread.currentThread().interrupt();
+ }
+ EnvironmentUtils.activeDaemon();
+
+ queryPlan = (QueryPlan) processor.parseSQLToPhysicalPlan("select * from root.isp.d1");
+ dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+ Assert.assertEquals(3, dataSet.getPaths().size());
+ while (dataSet.hasNext()) {
+ RowRecord record = dataSet.next();
+ Assert.assertEquals(6, record.getFields().size());
+ }
+ }
+
+ @Test
+ public void testInsertTabletSerialization() throws IllegalPathException, QueryProcessException {
+ InsertTabletPlan plan1 = getInsertTabletPlan();
+
+ PlanExecutor executor = new PlanExecutor();
+ executor.insertTablet(plan1);
+
+ ByteBuffer byteBuffer = ByteBuffer.allocate(10000);
+ plan1.serialize(byteBuffer);
+ byteBuffer.flip();
+
+ Assert.assertEquals(PhysicalPlanType.BATCHINSERT.ordinal(), byteBuffer.get());
+
+ InsertTabletPlan plan2 = new InsertTabletPlan();
+ plan2.deserialize(byteBuffer);
+
+ Assert.assertEquals(plan1, plan2);
+ }
+
+ @Test
+ public void testInsertTabletWithBitMapsSerialization()
+ throws IllegalPathException, QueryProcessException {
+ InsertTabletPlan plan1 = getInsertTabletPlan();
+ plan1.setBitMaps(new BitMap[6]);
+ BitMap[] bitMaps = plan1.getBitMaps();
+ for (int i = 0; i < 4; i++) {
+ if (bitMaps[i] == null) {
+ bitMaps[i] = new BitMap(4);
+ }
+ bitMaps[i].mark(i);
+ }
+ PlanExecutor executor = new PlanExecutor();
+ executor.insertTablet(plan1);
+
+ ByteBuffer byteBuffer = ByteBuffer.allocate(10000);
+ plan1.serialize(byteBuffer);
+ byteBuffer.flip();
+
+ Assert.assertEquals(PhysicalPlanType.BATCHINSERT.ordinal(), byteBuffer.get());
+
+ InsertTabletPlan plan2 = new InsertTabletPlan();
+ plan2.deserialize(byteBuffer);
+
+ Assert.assertEquals(plan1, plan2);
+ }
+
+ private InsertTabletPlan getInsertTabletPlan() throws IllegalPathException {
+ long[] times = new long[] {110L, 111L, 112L, 113L};
+ List<Integer> dataTypes = new ArrayList<>();
+ dataTypes.add(TSDataType.DOUBLE.ordinal());
+ dataTypes.add(TSDataType.FLOAT.ordinal());
+ dataTypes.add(TSDataType.INT64.ordinal());
+ dataTypes.add(TSDataType.INT32.ordinal());
+ dataTypes.add(TSDataType.BOOLEAN.ordinal());
+ dataTypes.add(TSDataType.TEXT.ordinal());
+
+ Object[] columns = new Object[6];
+ columns[0] = new double[4];
+ columns[1] = new float[4];
+ columns[2] = new long[4];
+ columns[3] = new int[4];
+ columns[4] = new boolean[4];
+ columns[5] = new Binary[4];
+
+ for (int r = 0; r < 4; r++) {
+ ((double[]) columns[0])[r] = 1.0;
+ ((float[]) columns[1])[r] = 2;
+ ((long[]) columns[2])[r] = 10000;
+ ((int[]) columns[3])[r] = 100;
+ ((boolean[]) columns[4])[r] = false;
+ ((Binary[]) columns[5])[r] = new Binary("hh" + r);
+ }
+
+ InsertTabletPlan tabletPlan =
+ new InsertTabletPlan(
+ new PartialPath("root.isp.d1"),
+ new String[] {"(s1,s2,s3)", "(s4,s5)", "s6"},
+ dataTypes);
+ tabletPlan.setTimes(times);
+ tabletPlan.setColumns(columns);
+ tabletPlan.setRowCount(times.length);
+ return tabletPlan;
+ }
}
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanSerializeTest.java b/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanSerializeTest.java
index d5c8e3d..e4f570e 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanSerializeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanSerializeTest.java
@@ -29,6 +29,7 @@
import org.apache.iotdb.db.qp.physical.PhysicalPlan.Factory;
import org.apache.iotdb.db.qp.physical.sys.AlterTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateMultiTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
import org.apache.iotdb.db.qp.physical.sys.DataAuthPlan;
@@ -190,6 +191,23 @@
}
@Test
+ public void createAlignedTimeSeriesPlanSerializeTest() throws IOException, IllegalPathException {
+ CreateAlignedTimeSeriesPlan createAlignedTimeSeriesPlan =
+ new CreateAlignedTimeSeriesPlan(
+ new PartialPath("root.sg.d1"),
+ Arrays.asList("s1", "s2"),
+ Arrays.asList(TSDataType.DOUBLE, TSDataType.INT32),
+ Arrays.asList(TSEncoding.RLE, TSEncoding.RLE),
+ CompressionType.SNAPPY,
+ null);
+
+ PhysicalPlan result = testTwoSerializeMethodAndDeserialize(createAlignedTimeSeriesPlan);
+
+ Assert.assertEquals(OperatorType.CREATE_ALIGNED_TIMESERIES, result.getOperatorType());
+ Assert.assertEquals(createAlignedTimeSeriesPlan, result);
+ }
+
+ @Test
public void createMuSerializeTest1() throws IOException, IllegalPathException {
CreateMultiTimeSeriesPlan plan = new CreateMultiTimeSeriesPlan();
plan.setPaths(
@@ -237,6 +255,32 @@
}
@Test
+ public void createMuSerializeTest3() throws IOException, IllegalPathException {
+ // same as:
+ // create timeseries root.sg.d1.s0 with datatype=DOUBLE, encoding=GORILLA, compression=SNAPPY
+ // create aligned timeseries root.sg.d1.(s1 INT64, s2 DOUBLE, s3 INT64)
+ // with encoding=(GORILLA, GORILLA, GORILLA), compression=SNAPPY
+ CreateMultiTimeSeriesPlan plan = new CreateMultiTimeSeriesPlan();
+ plan.setPaths(
+ Arrays.asList(new PartialPath("root.sg.d1.s0"), new PartialPath("root.sg.d1.(s1,s2,s3)")));
+ plan.setDataTypes(
+ Arrays.asList(TSDataType.DOUBLE, TSDataType.INT64, TSDataType.DOUBLE, TSDataType.INT64));
+ plan.setEncodings(
+ Arrays.asList(
+ TSEncoding.GORILLA, TSEncoding.GORILLA, TSEncoding.GORILLA, TSEncoding.GORILLA));
+ plan.setCompressors(Arrays.asList(CompressionType.SNAPPY, CompressionType.SNAPPY));
+ plan.setProps(null);
+ plan.setTags(null);
+ plan.setAttributes(null);
+ plan.setAlias(null);
+
+ PhysicalPlan result = testTwoSerializeMethodAndDeserialize(plan);
+
+ Assert.assertEquals(OperatorType.CREATE_MULTI_TIMESERIES, result.getOperatorType());
+ Assert.assertEquals(plan, result);
+ }
+
+ @Test
public void AlterTimeSeriesPlanSerializeTest() throws IOException, IllegalPathException {
AlterTimeSeriesPlan alterTimeSeriesPlan =
new AlterTimeSeriesPlan(
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanTest.java
index 916542b..7910c8c 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanTest.java
@@ -162,6 +162,21 @@
plan.toString());
}
+ // TODO @Steve SU
+ // @Test
+ // public void testMetadata4() throws QueryProcessException {
+ // String metadata =
+ // "create aligned timeseries root.vehicle.d1.(s1 INT32, s2 FLOAT) with encoding=(RLE, RLE)
+ // compression=SNAPPY";
+ // Planner processor = new Planner();
+ // CreateAlignedTimeSeriesPlan plan =
+ // (CreateAlignedTimeSeriesPlan) processor.parseSQLToPhysicalPlan(metadata);
+ // assertEquals(
+ // "devicePath: root.vehicle.d1, measurements: [s1, s2], dataTypes: [INT32, FLOAT],
+ // encoding: [RLE, RLE], compression: SNAPPY",
+ // plan.toString());
+ // }
+
@Test
public void testAuthor() throws QueryProcessException {
String sql =
diff --git a/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReaderTest.java b/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReaderTest.java
index 55eea5c..a77d592 100644
--- a/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReaderTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReaderTest.java
@@ -44,7 +44,9 @@
import java.util.List;
import java.util.Set;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
public class SeriesAggregateReaderTest {
@@ -71,7 +73,7 @@
PartialPath path = new PartialPath(SERIES_READER_TEST_SG + ".device0.sensor0");
Set<String> allSensors = new HashSet<>();
allSensors.add("sensor0");
- QueryDataSource queryDataSource = new QueryDataSource(path, seqResources, unseqResources);
+ QueryDataSource queryDataSource = new QueryDataSource(seqResources, unseqResources);
SeriesAggregateReader seriesReader =
new SeriesAggregateReader(
path,
diff --git a/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestampTest.java b/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestampTest.java
index d78d6d4..adb6b88 100644
--- a/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestampTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestampTest.java
@@ -61,11 +61,7 @@
@Test
public void test() throws IOException, IllegalPathException {
- QueryDataSource dataSource =
- new QueryDataSource(
- new PartialPath(SERIES_READER_TEST_SG + ".device0.sensor0"),
- seqResources,
- unseqResources);
+ QueryDataSource dataSource = new QueryDataSource(seqResources, unseqResources);
Set<String> allSensors = new HashSet<>();
allSensors.add("sensor0");
diff --git a/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderTestUtil.java b/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderTestUtil.java
index 4a699e0..2a9d808 100644
--- a/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderTestUtil.java
+++ b/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderTestUtil.java
@@ -37,6 +37,7 @@
import org.apache.iotdb.tsfile.write.TsFileWriter;
import org.apache.iotdb.tsfile.write.record.TSRecord;
import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import java.io.File;
@@ -167,7 +168,7 @@
List<String> deviceIds)
throws IOException, WriteProcessException {
TsFileWriter fileWriter = new TsFileWriter(tsFileResource.getTsFile());
- Map<String, MeasurementSchema> template = new HashMap<>();
+ Map<String, IMeasurementSchema> template = new HashMap<>();
for (MeasurementSchema measurementSchema : measurementSchemas) {
template.put(measurementSchema.getMeasurementId(), measurementSchema);
}
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/MemUtilsTest.java b/server/src/test/java/org/apache/iotdb/db/utils/MemUtilsTest.java
index 7468467..e236416 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/MemUtilsTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/MemUtilsTest.java
@@ -58,7 +58,7 @@
dataTypes.add(TSDataType.DOUBLE.ordinal());
sizeSum += 8 + TSDataType.DOUBLE.getDataTypeSize();
dataTypes.add(TSDataType.TEXT.ordinal());
- sizeSum += TSDataType.TEXT.getDataTypeSize();
+ sizeSum += 8 + TSDataType.TEXT.getDataTypeSize();
InsertTabletPlan insertPlan = new InsertTabletPlan(device, measurements, dataTypes);
Assert.assertEquals(sizeSum, MemUtils.getRecordSize(insertPlan, 0, 1, false));
}
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/VectorTVListTest.java b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/VectorTVListTest.java
new file mode 100644
index 0000000..c85021f
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/VectorTVListTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.utils.datastructure;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.BitMap;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class VectorTVListTest {
+
+ @Test
+ public void testVectorTVList1() {
+ List<TSDataType> dataTypes = new ArrayList<>();
+ for (int i = 0; i < 5; i++) {
+ dataTypes.add(TSDataType.INT64);
+ }
+ VectorTVList tvList = new VectorTVList(dataTypes);
+ for (long i = 0; i < 1000; i++) {
+ Object[] value = new Object[5];
+ for (int j = 0; j < 5; j++) {
+ value[j] = i;
+ }
+ tvList.putVector(i, value);
+ }
+ for (int i = 0; i < tvList.size; i++) {
+ StringBuilder builder = new StringBuilder("[");
+ builder.append(String.valueOf(i));
+ for (int j = 1; j < 5; j++) {
+ builder.append(", ").append(String.valueOf(i));
+ }
+ builder.append("]");
+ Assert.assertEquals(builder.toString(), tvList.getVector(i).toString());
+ Assert.assertEquals(i, tvList.getTime(i));
+ }
+ }
+
+ @Test
+ public void testVectorTVList2() {
+ List<TSDataType> dataTypes = new ArrayList<>();
+ dataTypes.add(TSDataType.BOOLEAN);
+ dataTypes.add(TSDataType.INT32);
+ dataTypes.add(TSDataType.INT64);
+ dataTypes.add(TSDataType.FLOAT);
+ dataTypes.add(TSDataType.DOUBLE);
+ dataTypes.add(TSDataType.TEXT);
+ VectorTVList tvList = new VectorTVList(dataTypes);
+ for (int i = 1000; i >= 0; i--) {
+ Object[] value = new Object[6];
+ value[0] = false;
+ value[1] = 100;
+ value[2] = 1000L;
+ value[3] = 0.1f;
+ value[4] = 0.2d;
+ value[5] = new Binary("Test");
+ tvList.putVector(i, value);
+ }
+ tvList.sort();
+ for (int i = 0; i < tvList.size; i++) {
+ StringBuilder builder = new StringBuilder("[");
+ builder.append("false, 100, 1000, 0.1, 0.2, Test");
+ builder.append("]");
+ Assert.assertEquals(builder.toString(), tvList.getVector(i).toString());
+ Assert.assertEquals(i, tvList.getTime(i));
+ }
+ }
+
+ @Test
+ public void testVectorTVLists() {
+ List<TSDataType> dataTypes = new ArrayList<>();
+ for (int i = 0; i < 5; i++) {
+ dataTypes.add(TSDataType.INT64);
+ }
+ VectorTVList tvList = new VectorTVList(dataTypes);
+ long[][] vectorArray = new long[5][1001];
+ List<Long> timeList = new ArrayList<>();
+ for (int i = 1000; i >= 0; i--) {
+ timeList.add((long) i);
+ for (int j = 0; j < 5; j++) {
+ vectorArray[j][i] = (long) i;
+ }
+ }
+
+ tvList.putVectors(
+ ArrayUtils.toPrimitive(timeList.toArray(new Long[0])), null, vectorArray, 0, 1000);
+ for (long i = 0; i < tvList.size; i++) {
+ Assert.assertEquals(tvList.size - i, tvList.getTime((int) i));
+ }
+ }
+
+ @Test
+ public void testVectorTVListsWithBitMaps() {
+ List<TSDataType> dataTypes = new ArrayList<>();
+ BitMap[] bitMaps = new BitMap[5];
+ for (int i = 0; i < 5; i++) {
+ dataTypes.add(TSDataType.INT64);
+ bitMaps[i] = new BitMap(1001);
+ }
+ VectorTVList tvList = new VectorTVList(dataTypes);
+ long[][] vectorArray = new long[5][1001];
+ List<Long> timeList = new ArrayList<>();
+ for (int i = 1000; i >= 0; i--) {
+ timeList.add((long) i);
+ for (int j = 0; j < 5; j++) {
+ vectorArray[j][i] = (long) i;
+ if (i % 100 == 0) {
+ bitMaps[j].mark(i);
+ }
+ }
+ }
+
+ tvList.putVectors(
+ ArrayUtils.toPrimitive(timeList.toArray(new Long[0])), bitMaps, vectorArray, 0, 1000);
+ for (long i = 0; i < tvList.size; i++) {
+ Assert.assertEquals(tvList.size - i, tvList.getTime((int) i));
+ if (i % 100 == 0) {
+ Assert.assertEquals("[null, null, null, null, null]", tvList.getVector((int) i).toString());
+ }
+ }
+ }
+}
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/recover/LogReplayerTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/recover/LogReplayerTest.java
index 7e58c2f..5dcb1b7 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/recover/LogReplayerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/recover/LogReplayerTest.java
@@ -44,10 +44,12 @@
import org.apache.iotdb.db.writelog.manager.MultiFileLogNodeManager;
import org.apache.iotdb.db.writelog.node.WriteLogNode;
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.read.reader.IPointReader;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.junit.After;
import org.junit.Before;
@@ -164,9 +166,12 @@
memTable.query(
"root.sg.device" + i,
"sensor" + i,
- TSDataType.INT64,
- TSEncoding.RLE,
- Collections.emptyMap(),
+ new MeasurementSchema(
+ "sensor" + i,
+ TSDataType.INT64,
+ TSEncoding.RLE,
+ CompressionType.UNCOMPRESSED,
+ Collections.emptyMap()),
Long.MIN_VALUE,
null);
IPointReader iterator = memChunk.getPointReader();
@@ -200,9 +205,12 @@
memTable.query(
"root.sg.device5",
"sensor" + i,
- TSDataType.INT64,
- TSEncoding.PLAIN,
- Collections.emptyMap(),
+ new MeasurementSchema(
+ "sensor" + i,
+ TSDataType.INT64,
+ TSEncoding.PLAIN,
+ CompressionType.UNCOMPRESSED,
+ Collections.emptyMap()),
Long.MIN_VALUE,
null);
// s0 has datatype boolean, but required INT64, will return null
@@ -254,7 +262,7 @@
MeasurementMNode[] mNodes = new MeasurementMNode[2];
mNodes[0] = new MeasurementMNode(null, "sensor0", null, null);
- mNodes[0] = new MeasurementMNode(null, "sensor1", null, null);
+ mNodes[1] = new MeasurementMNode(null, "sensor1", null, null);
InsertTabletPlan insertTabletPlan =
new InsertTabletPlan(new PartialPath(deviceId), measurements, dataTypes);
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java
index 0dae987..f24f023 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java
@@ -48,6 +48,7 @@
import org.apache.iotdb.tsfile.write.TsFileWriter;
import org.apache.iotdb.tsfile.write.record.TSRecord;
import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.Schema;
import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
@@ -113,7 +114,7 @@
}
Schema schema = new Schema();
- Map<String, MeasurementSchema> template = new HashMap<>();
+ Map<String, IMeasurementSchema> template = new HashMap<>();
for (int i = 0; i < 10; i++) {
template.put(
"sensor" + i, new MeasurementSchema("sensor" + i, TSDataType.INT64, TSEncoding.PLAIN));
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java
index 92dec08..77f7940 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java
@@ -40,6 +40,7 @@
import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
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;
@@ -258,8 +259,8 @@
PriorityMergeReader unSeqMergeReader = new PriorityMergeReader();
int priorityValue = 1;
- for (ChunkMetadata chunkMetaData : metadataQuerier.getChunkMetaDataList(path)) {
- Chunk chunk = chunkLoader.loadChunk(chunkMetaData);
+ for (IChunkMetadata chunkMetaData : metadataQuerier.getChunkMetaDataList(path)) {
+ Chunk chunk = chunkLoader.loadChunk((ChunkMetadata) chunkMetaData);
ChunkReader chunkReader = new ChunkReader(chunk, null);
unSeqMergeReader.addReader(new ChunkDataIterator(chunkReader), priorityValue++);
}
diff --git a/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java
index f2fd3c1..e5fd65a 100644
--- a/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java
+++ b/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java
@@ -46,6 +46,10 @@
LOAD_FILE_ERROR(316),
STORAGE_GROUP_NOT_READY(317),
ILLEGAL_PARAMETER(318),
+ ALIGNED_TIMESERIES_ERROR(319),
+ DUPLICATED_TEMPLATE(320),
+ UNDEFINED_TEMPLATE(321),
+ STORAGE_GROUP_NOT_EXIST(322),
EXECUTE_STATEMENT_ERROR(400),
SQL_PARSE_ERROR(401),
diff --git a/session/src/main/java/org/apache/iotdb/session/Session.java b/session/src/main/java/org/apache/iotdb/session/Session.java
index b3d1f95..6fc4ab4 100644
--- a/session/src/main/java/org/apache/iotdb/session/Session.java
+++ b/session/src/main/java/org/apache/iotdb/session/Session.java
@@ -23,6 +23,8 @@
import org.apache.iotdb.rpc.RedirectException;
import org.apache.iotdb.rpc.StatementExecutionException;
import org.apache.iotdb.service.rpc.thrift.EndPoint;
+import org.apache.iotdb.service.rpc.thrift.TSCreateAlignedTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateDeviceTemplateReq;
import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
@@ -34,14 +36,17 @@
import org.apache.iotdb.service.rpc.thrift.TSInsertTabletReq;
import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
import org.apache.iotdb.service.rpc.thrift.TSProtocolVersion;
+import org.apache.iotdb.service.rpc.thrift.TSSetDeviceTemplateReq;
import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
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.utils.Binary;
+import org.apache.iotdb.tsfile.utils.BitMap;
import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.slf4j.Logger;
@@ -58,6 +63,7 @@
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
@SuppressWarnings({"java:S107", "java:S1135"}) // need enough parameters, ignore todos
public class Session {
@@ -71,7 +77,7 @@
protected String username;
protected String password;
protected int fetchSize;
-
+ private static final byte TYPE_NULL = -2;
/**
* Timeout of query can be set by users. If not set, default value 0 will be used, which will use
* server configuration.
@@ -371,6 +377,37 @@
return request;
}
+ public void createAlignedTimeseries(
+ String devicePath,
+ List<String> measurements,
+ List<TSDataType> dataTypes,
+ List<TSEncoding> encodings,
+ CompressionType compressor,
+ List<String> measurementAliasList)
+ throws IoTDBConnectionException, StatementExecutionException {
+ TSCreateAlignedTimeseriesReq request =
+ getTSCreateAlignedTimeseriesReq(
+ devicePath, measurements, dataTypes, encodings, compressor, measurementAliasList);
+ defaultSessionConnection.createAlignedTimeseries(request);
+ }
+
+ private TSCreateAlignedTimeseriesReq getTSCreateAlignedTimeseriesReq(
+ String devicePath,
+ List<String> measurements,
+ List<TSDataType> dataTypes,
+ List<TSEncoding> encodings,
+ CompressionType compressor,
+ List<String> measurementAliasList) {
+ TSCreateAlignedTimeseriesReq request = new TSCreateAlignedTimeseriesReq();
+ request.setDevicePath(devicePath);
+ request.setMeasurements(measurements);
+ request.setDataTypes(dataTypes.stream().map(TSDataType::ordinal).collect(Collectors.toList()));
+ request.setEncodings(encodings.stream().map(TSEncoding::ordinal).collect(Collectors.toList()));
+ request.setCompressor(compressor.ordinal());
+ request.setMeasurementAlias(measurementAliasList);
+ return request;
+ }
+
public void createMultiTimeseries(
List<String> paths,
List<TSDataType> dataTypes,
@@ -407,13 +444,13 @@
request.setPaths(paths);
- List<Integer> dataTypeOrdinals = new ArrayList<>(paths.size());
+ List<Integer> dataTypeOrdinals = new ArrayList<>(dataTypes.size());
for (TSDataType dataType : dataTypes) {
dataTypeOrdinals.add(dataType.ordinal());
}
request.setDataTypes(dataTypeOrdinals);
- List<Integer> encodingOrdinals = new ArrayList<>(paths.size());
+ List<Integer> encodingOrdinals = new ArrayList<>(dataTypes.size());
for (TSEncoding encoding : encodings) {
encodingOrdinals.add(encoding.ordinal());
}
@@ -1087,9 +1124,24 @@
TSInsertTabletReq request = new TSInsertTabletReq();
request.setDeviceId(tablet.deviceId);
- for (MeasurementSchema measurementSchema : tablet.getSchemas()) {
- request.addToMeasurements(measurementSchema.getMeasurementId());
- request.addToTypes(measurementSchema.getType().ordinal());
+ for (IMeasurementSchema measurementSchema : tablet.getSchemas()) {
+ if (measurementSchema instanceof MeasurementSchema) {
+ request.addToMeasurements(measurementSchema.getMeasurementId());
+ request.addToTypes(measurementSchema.getType().ordinal());
+ } else {
+ int measurementsSize = measurementSchema.getValueMeasurementIdList().size();
+ StringBuilder measurement = new StringBuilder("(");
+ for (int i = 0; i < measurementsSize; i++) {
+ measurement.append(measurementSchema.getValueMeasurementIdList().get(i));
+ if (i != measurementsSize - 1) {
+ measurement.append(",");
+ } else {
+ measurement.append(")");
+ }
+ request.addToTypes(measurementSchema.getValueTSDataTypeList().get(i).ordinal());
+ }
+ request.addToMeasurements(measurement.toString());
+ }
}
request.setTimestamps(SessionUtils.getTimeBuffer(tablet));
request.setValues(SessionUtils.getValueBuffer(tablet));
@@ -1188,7 +1240,7 @@
request.addToDeviceIds(tablet.deviceId);
List<String> measurements = new ArrayList<>();
List<Integer> dataTypes = new ArrayList<>();
- for (MeasurementSchema measurementSchema : tablet.getSchemas()) {
+ for (IMeasurementSchema measurementSchema : tablet.getSchemas()) {
measurements.add(measurementSchema.getMeasurementId());
dataTypes.add(measurementSchema.getType().ordinal());
}
@@ -1402,6 +1454,10 @@
private void putValues(List<TSDataType> types, List<Object> values, ByteBuffer buffer)
throws IoTDBConnectionException {
for (int i = 0; i < values.size(); i++) {
+ if (values.get(i) == null) {
+ ReadWriteIOUtils.write(TYPE_NULL, buffer);
+ continue;
+ }
ReadWriteIOUtils.write(types.get(i), buffer);
switch (types.get(i)) {
case BOOLEAN:
@@ -1472,8 +1528,26 @@
}
Arrays.sort(index, Comparator.comparingLong(o -> tablet.timestamps[o]));
Arrays.sort(tablet.timestamps, 0, tablet.rowSize);
+ int columnIndex = 0;
for (int i = 0; i < tablet.getSchemas().size(); i++) {
- tablet.values[i] = sortList(tablet.values[i], tablet.getSchemas().get(i).getType(), index);
+ IMeasurementSchema schema = tablet.getSchemas().get(i);
+ if (schema instanceof MeasurementSchema) {
+ tablet.values[columnIndex] = sortList(tablet.values[columnIndex], schema.getType(), index);
+ if (tablet.bitMaps != null && tablet.bitMaps[columnIndex] != null) {
+ tablet.bitMaps[columnIndex] = sortBitMap(tablet.bitMaps[columnIndex], index);
+ }
+ columnIndex++;
+ } else {
+ int measurementSize = schema.getValueMeasurementIdList().size();
+ for (int j = 0; j < measurementSize; j++) {
+ tablet.values[columnIndex] =
+ sortList(tablet.values[columnIndex], schema.getValueTSDataTypeList().get(j), index);
+ if (tablet.bitMaps != null && tablet.bitMaps[columnIndex] != null) {
+ tablet.bitMaps[columnIndex] = sortBitMap(tablet.bitMaps[columnIndex], index);
+ }
+ columnIndex++;
+ }
+ }
}
}
@@ -1534,6 +1608,90 @@
}
}
+ /**
+ * sort BitMap by index
+ *
+ * @param bitMap BitMap to be sorted
+ * @param index index
+ * @return sorted bitMap
+ */
+ private BitMap sortBitMap(BitMap bitMap, Integer[] index) {
+ BitMap sortedBitMap = new BitMap(bitMap.getSize());
+ for (int i = 0; i < index.length; i++) {
+ if (bitMap.isMarked(index[i])) {
+ sortedBitMap.mark(i);
+ }
+ }
+ return sortedBitMap;
+ }
+
+ public void setDeviceTemplate(String templateName, String prefixPath)
+ throws IoTDBConnectionException, StatementExecutionException {
+ TSSetDeviceTemplateReq request = getTSSetDeviceTemplateReq(templateName, prefixPath);
+ defaultSessionConnection.setDeviceTemplate(request);
+ }
+
+ /**
+ * @param name template name
+ * @param measurements List of measurements, if it is a single measurement, just put it's name
+ * into a list and add to measurements if it is a vector measurement, put all measurements of
+ * the vector into a list and add to measurements
+ * @param dataTypes List of datatypes, if it is a single measurement, just put it's type into a
+ * list and add to dataTypes if it is a vector measurement, put all types of the vector into a
+ * list and add to dataTypes
+ * @param encodings List of encodings, if it is a single measurement, just put it's encoding into
+ * a list and add to encodings if it is a vector measurement, put all encodings of the vector
+ * into a list and add to encodings
+ * @param compressors List of compressors
+ * @throws IoTDBConnectionException
+ * @throws StatementExecutionException
+ */
+ public void createDeviceTemplate(
+ String name,
+ List<List<String>> measurements,
+ List<List<TSDataType>> dataTypes,
+ List<List<TSEncoding>> encodings,
+ List<CompressionType> compressors)
+ throws IoTDBConnectionException, StatementExecutionException {
+ TSCreateDeviceTemplateReq request =
+ getTSCreateDeviceTemplateReq(name, measurements, dataTypes, encodings, compressors);
+ defaultSessionConnection.createDeviceTemplate(request);
+ }
+
+ private TSSetDeviceTemplateReq getTSSetDeviceTemplateReq(String templateName, String prefixPath) {
+ TSSetDeviceTemplateReq request = new TSSetDeviceTemplateReq();
+ request.setTemplateName(templateName);
+ request.setPrefixPath(prefixPath);
+ return request;
+ }
+
+ private TSCreateDeviceTemplateReq getTSCreateDeviceTemplateReq(
+ String name,
+ List<List<String>> measurements,
+ List<List<TSDataType>> dataTypes,
+ List<List<TSEncoding>> encodings,
+ List<CompressionType> compressors) {
+ TSCreateDeviceTemplateReq request = new TSCreateDeviceTemplateReq();
+ request.setName(name);
+ request.setMeasurements(measurements);
+
+ List<List<Integer>> requestType = new ArrayList<>();
+ for (List<TSDataType> typesList : dataTypes) {
+ requestType.add(typesList.stream().map(TSDataType::ordinal).collect(Collectors.toList()));
+ }
+ request.setDataTypes(requestType);
+
+ List<List<Integer>> requestEncoding = new ArrayList<>();
+ for (List<TSEncoding> encodingList : encodings) {
+ requestEncoding.add(
+ encodingList.stream().map(TSEncoding::ordinal).collect(Collectors.toList()));
+ }
+ request.setEncodings(requestEncoding);
+ request.setCompressors(
+ compressors.stream().map(CompressionType::ordinal).collect(Collectors.toList()));
+ return request;
+ }
+
public boolean isEnableQueryRedirection() {
return enableQueryRedirection;
}
diff --git a/session/src/main/java/org/apache/iotdb/session/SessionConnection.java b/session/src/main/java/org/apache/iotdb/session/SessionConnection.java
index ef04497..17e0a10 100644
--- a/session/src/main/java/org/apache/iotdb/session/SessionConnection.java
+++ b/session/src/main/java/org/apache/iotdb/session/SessionConnection.java
@@ -26,6 +26,8 @@
import org.apache.iotdb.rpc.StatementExecutionException;
import org.apache.iotdb.service.rpc.thrift.EndPoint;
import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateAlignedTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateDeviceTemplateReq;
import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq;
@@ -42,6 +44,7 @@
import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq;
import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp;
import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
+import org.apache.iotdb.service.rpc.thrift.TSSetDeviceTemplateReq;
import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
import org.apache.iotdb.service.rpc.thrift.TSStatus;
@@ -232,6 +235,25 @@
}
}
+ protected void createAlignedTimeseries(TSCreateAlignedTimeseriesReq request)
+ throws IoTDBConnectionException, StatementExecutionException {
+ request.setSessionId(sessionId);
+ try {
+ RpcUtils.verifySuccess(client.createAlignedTimeseries(request));
+ } catch (TException e) {
+ if (reconnect()) {
+ try {
+ request.setSessionId(sessionId);
+ RpcUtils.verifySuccess(client.createAlignedTimeseries(request));
+ } catch (TException tException) {
+ throw new IoTDBConnectionException(tException);
+ }
+ } else {
+ throw new IoTDBConnectionException(MSG_RECONNECTION_FAIL);
+ }
+ }
+ }
+
protected void createMultiTimeseries(TSCreateMultiTimeseriesReq request)
throws IoTDBConnectionException, StatementExecutionException {
request.setSessionId(sessionId);
@@ -671,6 +693,44 @@
return flag;
}
+ protected void createDeviceTemplate(TSCreateDeviceTemplateReq request)
+ throws IoTDBConnectionException, StatementExecutionException {
+ request.setSessionId(sessionId);
+ try {
+ RpcUtils.verifySuccess(client.createDeviceTemplate(request));
+ } catch (TException e) {
+ if (reconnect()) {
+ try {
+ request.setSessionId(sessionId);
+ RpcUtils.verifySuccess(client.createDeviceTemplate(request));
+ } catch (TException tException) {
+ throw new IoTDBConnectionException(tException);
+ }
+ } else {
+ throw new IoTDBConnectionException(MSG_RECONNECTION_FAIL);
+ }
+ }
+ }
+
+ protected void setDeviceTemplate(TSSetDeviceTemplateReq request)
+ throws IoTDBConnectionException, StatementExecutionException {
+ request.setSessionId(sessionId);
+ try {
+ RpcUtils.verifySuccess(client.setDeviceTemplate(request));
+ } catch (TException e) {
+ if (reconnect()) {
+ try {
+ request.setSessionId(sessionId);
+ RpcUtils.verifySuccess(client.setDeviceTemplate(request));
+ } catch (TException tException) {
+ throw new IoTDBConnectionException(tException);
+ }
+ } else {
+ throw new IoTDBConnectionException(MSG_RECONNECTION_FAIL);
+ }
+ }
+ }
+
public boolean isEnableRedirect() {
return enableRedirect;
}
diff --git a/session/src/main/java/org/apache/iotdb/session/SessionUtils.java b/session/src/main/java/org/apache/iotdb/session/SessionUtils.java
index a2938c0..f028990 100644
--- a/session/src/main/java/org/apache/iotdb/session/SessionUtils.java
+++ b/session/src/main/java/org/apache/iotdb/session/SessionUtils.java
@@ -21,8 +21,11 @@
import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
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.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import java.nio.ByteBuffer;
@@ -39,53 +42,111 @@
@SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
public static ByteBuffer getValueBuffer(Tablet tablet) {
- ByteBuffer valueBuffer = ByteBuffer.allocate(tablet.getValueBytesSize());
+ ByteBuffer valueBuffer = ByteBuffer.allocate(tablet.getTotalValueOccupation());
+ int indexOfValues = 0;
for (int i = 0; i < tablet.getSchemas().size(); i++) {
- TSDataType dataType = tablet.getSchemas().get(i).getType();
- switch (dataType) {
- case INT32:
- int[] intValues = (int[]) tablet.values[i];
- for (int index = 0; index < tablet.rowSize; index++) {
- valueBuffer.putInt(intValues[index]);
+ IMeasurementSchema schema = tablet.getSchemas().get(i);
+ if (schema instanceof MeasurementSchema) {
+ getValueBufferOfDataType(schema.getType(), tablet, indexOfValues, valueBuffer);
+ indexOfValues++;
+ } else {
+ for (int j = 0; j < schema.getValueTSDataTypeList().size(); j++) {
+ getValueBufferOfDataType(
+ schema.getValueTSDataTypeList().get(j), tablet, indexOfValues, valueBuffer);
+ indexOfValues++;
+ }
+ }
+ }
+ if (tablet.bitMaps != null) {
+ for (BitMap bitMap : tablet.bitMaps) {
+ boolean columnHasNull = bitMap != null && !bitMap.isAllUnmarked();
+ valueBuffer.put(BytesUtils.boolToByte(columnHasNull));
+ if (columnHasNull) {
+ byte[] bytes = bitMap.getByteArray();
+ for (int j = 0; j < tablet.rowSize / Byte.SIZE + 1; j++) {
+ valueBuffer.put(bytes[j]);
}
- break;
- case INT64:
- long[] longValues = (long[]) tablet.values[i];
- for (int index = 0; index < tablet.rowSize; index++) {
- valueBuffer.putLong(longValues[index]);
- }
- break;
- case FLOAT:
- float[] floatValues = (float[]) tablet.values[i];
- for (int index = 0; index < tablet.rowSize; index++) {
- valueBuffer.putFloat(floatValues[index]);
- }
- break;
- case DOUBLE:
- double[] doubleValues = (double[]) tablet.values[i];
- for (int index = 0; index < tablet.rowSize; index++) {
- valueBuffer.putDouble(doubleValues[index]);
- }
- break;
- case BOOLEAN:
- boolean[] boolValues = (boolean[]) tablet.values[i];
- for (int index = 0; index < tablet.rowSize; index++) {
- valueBuffer.put(BytesUtils.boolToByte(boolValues[index]));
- }
- break;
- case TEXT:
- Binary[] binaryValues = (Binary[]) tablet.values[i];
- for (int index = 0; index < tablet.rowSize; index++) {
- valueBuffer.putInt(binaryValues[index].getLength());
- valueBuffer.put(binaryValues[index].getValues());
- }
- break;
- default:
- throw new UnSupportedDataTypeException(
- String.format("Data type %s is not supported.", dataType));
+ }
}
}
valueBuffer.flip();
return valueBuffer;
}
+
+ private static void getValueBufferOfDataType(
+ TSDataType dataType, Tablet tablet, int i, ByteBuffer valueBuffer) {
+
+ switch (dataType) {
+ case INT32:
+ int[] intValues = (int[]) tablet.values[i];
+ for (int index = 0; index < tablet.rowSize; index++) {
+ if (tablet.bitMaps == null
+ || tablet.bitMaps[i] == null
+ || !tablet.bitMaps[i].isMarked(index)) {
+ valueBuffer.putInt(intValues[index]);
+ } else {
+ valueBuffer.putInt(Integer.MIN_VALUE);
+ }
+ }
+ break;
+ case INT64:
+ long[] longValues = (long[]) tablet.values[i];
+ for (int index = 0; index < tablet.rowSize; index++) {
+ if (tablet.bitMaps == null
+ || tablet.bitMaps[i] == null
+ || !tablet.bitMaps[i].isMarked(index)) {
+ valueBuffer.putLong(longValues[index]);
+ } else {
+ valueBuffer.putLong(Long.MIN_VALUE);
+ }
+ }
+ break;
+ case FLOAT:
+ float[] floatValues = (float[]) tablet.values[i];
+ for (int index = 0; index < tablet.rowSize; index++) {
+ if (tablet.bitMaps == null
+ || tablet.bitMaps[i] == null
+ || !tablet.bitMaps[i].isMarked(index)) {
+ valueBuffer.putFloat(floatValues[index]);
+ } else {
+ valueBuffer.putFloat(Float.MIN_VALUE);
+ }
+ }
+ break;
+ case DOUBLE:
+ double[] doubleValues = (double[]) tablet.values[i];
+ for (int index = 0; index < tablet.rowSize; index++) {
+ if (tablet.bitMaps == null
+ || tablet.bitMaps[i] == null
+ || !tablet.bitMaps[i].isMarked(index)) {
+ valueBuffer.putDouble(doubleValues[index]);
+ } else {
+ valueBuffer.putDouble(Double.MIN_VALUE);
+ }
+ }
+ break;
+ case BOOLEAN:
+ boolean[] boolValues = (boolean[]) tablet.values[i];
+ for (int index = 0; index < tablet.rowSize; index++) {
+ if (tablet.bitMaps == null
+ || tablet.bitMaps[i] == null
+ || !tablet.bitMaps[i].isMarked(index)) {
+ valueBuffer.put(BytesUtils.boolToByte(boolValues[index]));
+ } else {
+ valueBuffer.put(BytesUtils.boolToByte(false));
+ }
+ }
+ break;
+ case TEXT:
+ Binary[] binaryValues = (Binary[]) tablet.values[i];
+ for (int index = 0; index < tablet.rowSize; index++) {
+ valueBuffer.putInt(binaryValues[index].getLength());
+ valueBuffer.put(binaryValues[index].getValues());
+ }
+ break;
+ default:
+ throw new UnSupportedDataTypeException(
+ String.format("Data type %s is not supported.", dataType));
+ }
+ }
}
diff --git a/session/src/test/java/org/apache/iotdb/session/IoTDBSessionComplexIT.java b/session/src/test/java/org/apache/iotdb/session/IoTDBSessionComplexIT.java
index f5016f3..281c669 100644
--- a/session/src/test/java/org/apache/iotdb/session/IoTDBSessionComplexIT.java
+++ b/session/src/test/java/org/apache/iotdb/session/IoTDBSessionComplexIT.java
@@ -33,6 +33,7 @@
import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import org.apache.iotdb.tsfile.read.common.Field;
import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.junit.After;
@@ -216,7 +217,7 @@
createTimeseries();
- List<MeasurementSchema> schemaList = new ArrayList<>();
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
schemaList.add(new MeasurementSchema("s1", TSDataType.INT64, TSEncoding.RLE));
schemaList.add(new MeasurementSchema("s2", TSDataType.INT64, TSEncoding.RLE));
schemaList.add(new MeasurementSchema("s3", TSDataType.INT64, TSEncoding.RLE));
@@ -475,7 +476,7 @@
session.insertRecord(deviceId, time, measurements, types, values);
}
- List<MeasurementSchema> schemaList = new ArrayList<>();
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
schemaList.add(new MeasurementSchema("s1", TSDataType.INT64, TSEncoding.RLE));
schemaList.add(new MeasurementSchema("s2", TSDataType.INT64, TSEncoding.RLE));
schemaList.add(new MeasurementSchema("s3", TSDataType.INT64, TSEncoding.RLE));
@@ -596,7 +597,7 @@
private void insertTablet(String deviceId)
throws IoTDBConnectionException, StatementExecutionException {
- List<MeasurementSchema> schemaList = new ArrayList<>();
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
schemaList.add(new MeasurementSchema("s1", TSDataType.INT64, TSEncoding.RLE));
schemaList.add(new MeasurementSchema("s2", TSDataType.INT64, TSEncoding.RLE));
schemaList.add(new MeasurementSchema("s3", TSDataType.INT64, TSEncoding.RLE));
diff --git a/session/src/test/java/org/apache/iotdb/session/IoTDBSessionSimpleIT.java b/session/src/test/java/org/apache/iotdb/session/IoTDBSessionSimpleIT.java
index ed72527..7e0ed11 100644
--- a/session/src/test/java/org/apache/iotdb/session/IoTDBSessionSimpleIT.java
+++ b/session/src/test/java/org/apache/iotdb/session/IoTDBSessionSimpleIT.java
@@ -35,7 +35,9 @@
import org.apache.iotdb.tsfile.read.common.RowRecord;
import org.apache.iotdb.tsfile.utils.Binary;
import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
import org.junit.After;
import org.junit.Assert;
@@ -44,6 +46,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.security.SecureRandom;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
@@ -109,7 +112,7 @@
session = new Session("127.0.0.1", 6667, "root", "root");
session.open();
- List<MeasurementSchema> schemaList = new ArrayList<>();
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
schemaList.add(new MeasurementSchema("s1", TSDataType.INT64));
schemaList.add(new MeasurementSchema("s2", TSDataType.DOUBLE));
schemaList.add(new MeasurementSchema("s3", TSDataType.TEXT));
@@ -370,6 +373,58 @@
}
@Test
+ public void testInsertTabletWithAlignedTimeseries()
+ throws IoTDBConnectionException, StatementExecutionException {
+ session = new Session("127.0.0.1", 6667, "root", "root");
+ session.open();
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
+ schemaList.add(new MeasurementSchema("s0", TSDataType.INT64));
+ schemaList.add(
+ new VectorMeasurementSchema(
+ new String[] {"s1", "s2", "s3"},
+ new TSDataType[] {TSDataType.INT64, TSDataType.INT32, TSDataType.TEXT}));
+ schemaList.add(new MeasurementSchema("s4", TSDataType.INT32));
+
+ Tablet tablet = new Tablet("root.sg1.d1", schemaList);
+ long timestamp = System.currentTimeMillis();
+
+ for (long row = 0; row < 10; row++) {
+ int rowIndex = tablet.rowSize++;
+ tablet.addTimestamp(rowIndex, timestamp);
+ tablet.addValue(
+ schemaList.get(0).getMeasurementId(), rowIndex, new SecureRandom().nextLong());
+ tablet.addValue(
+ schemaList.get(1).getValueMeasurementIdList().get(0),
+ rowIndex,
+ new SecureRandom().nextLong());
+ tablet.addValue(
+ schemaList.get(1).getValueMeasurementIdList().get(1),
+ rowIndex,
+ new SecureRandom().nextInt());
+ tablet.addValue(
+ schemaList.get(1).getValueMeasurementIdList().get(2), rowIndex, new Binary("test"));
+ tablet.addValue(schemaList.get(2).getMeasurementId(), rowIndex, new SecureRandom().nextInt());
+ timestamp++;
+ }
+
+ if (tablet.rowSize != 0) {
+ session.insertTablet(tablet);
+ tablet.reset();
+ }
+
+ SessionDataSet dataSet = session.executeQueryStatement("select count(*) from root");
+ while (dataSet.hasNext()) {
+ RowRecord rowRecord = dataSet.next();
+ Assert.assertEquals(10L, rowRecord.getFields().get(0).getLongV());
+ Assert.assertEquals(10L, rowRecord.getFields().get(1).getLongV());
+ Assert.assertEquals(10L, rowRecord.getFields().get(2).getLongV());
+ Assert.assertEquals(10L, rowRecord.getFields().get(3).getLongV());
+ Assert.assertEquals(10L, rowRecord.getFields().get(4).getLongV());
+ }
+ session.close();
+ }
+
+ @Test
public void createTimeSeriesWithDoubleTicks()
throws IoTDBConnectionException, StatementExecutionException {
session = new Session("127.0.0.1", 6667, "root", "root");
diff --git a/session/src/test/java/org/apache/iotdb/session/SessionCacheLeaderUT.java b/session/src/test/java/org/apache/iotdb/session/SessionCacheLeaderUT.java
index 4b52005..763a27b 100644
--- a/session/src/test/java/org/apache/iotdb/session/SessionCacheLeaderUT.java
+++ b/session/src/test/java/org/apache/iotdb/session/SessionCacheLeaderUT.java
@@ -32,6 +32,7 @@
import org.apache.iotdb.service.rpc.thrift.TSInsertTabletsReq;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.junit.Test;
@@ -532,7 +533,7 @@
assertNull(session.endPointToSessionConnection);
String deviceId = "root.sg2.d2";
- List<MeasurementSchema> schemaList = new ArrayList<>();
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
schemaList.add(new MeasurementSchema("s1", TSDataType.INT64));
schemaList.add(new MeasurementSchema("s2", TSDataType.INT64));
schemaList.add(new MeasurementSchema("s3", TSDataType.INT64));
@@ -614,7 +615,7 @@
add("root.sg4.d1");
}
};
- List<MeasurementSchema> schemaList = new ArrayList<>();
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
schemaList.add(new MeasurementSchema("s1", TSDataType.INT64));
schemaList.add(new MeasurementSchema("s2", TSDataType.INT64));
schemaList.add(new MeasurementSchema("s3", TSDataType.INT64));
diff --git a/session/src/test/java/org/apache/iotdb/session/SessionUT.java b/session/src/test/java/org/apache/iotdb/session/SessionUT.java
index 26d0ddd..59d9f5b 100644
--- a/session/src/test/java/org/apache/iotdb/session/SessionUT.java
+++ b/session/src/test/java/org/apache/iotdb/session/SessionUT.java
@@ -28,6 +28,7 @@
import org.apache.iotdb.tsfile.read.common.RowRecord;
import org.apache.iotdb.tsfile.utils.Binary;
import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.junit.After;
@@ -37,6 +38,7 @@
import java.time.ZoneId;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
import static org.junit.Assert.assertArrayEquals;
@@ -69,7 +71,7 @@
!!!
*/
session = new Session("127.0.0.1", 6667, "root", "root", null);
- List<MeasurementSchema> schemaList = new ArrayList<>();
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
schemaList.add(new MeasurementSchema("s1", TSDataType.INT64, TSEncoding.RLE));
// insert three rows data
Tablet tablet = new Tablet("root.sg1.d1", schemaList, 3);
@@ -135,7 +137,7 @@
session.createTimeseries(
deviceId + ".s4", TSDataType.DOUBLE, TSEncoding.RLE, CompressionType.UNCOMPRESSED);
- List<MeasurementSchema> schemaList = new ArrayList<>();
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
schemaList.add(new MeasurementSchema("s1", TSDataType.INT64, TSEncoding.RLE));
schemaList.add(new MeasurementSchema("s2", TSDataType.DOUBLE, TSEncoding.RLE));
schemaList.add(new MeasurementSchema("s3", TSDataType.TEXT, TSEncoding.PLAIN));
@@ -199,4 +201,50 @@
session.setTimeout(60000);
Assert.assertEquals(60000, session.getTimeout());
}
+
+ @Test
+ public void setDeviceTemplate() throws IoTDBConnectionException, StatementExecutionException {
+ session = new Session("127.0.0.1", 6667, "root", "root", ZoneId.of("+05:00"));
+ session.open();
+
+ session.setDeviceTemplate("template1", "root.sg.1");
+ }
+
+ @Test
+ public void createDeviceTemplate() throws IoTDBConnectionException, StatementExecutionException {
+ session = new Session("127.0.0.1", 6667, "root", "root", ZoneId.of("+05:00"));
+ session.open();
+
+ List<List<String>> measurementList = new ArrayList<>();
+ measurementList.add(Collections.singletonList("s11"));
+ List<String> measurements = new ArrayList<>();
+ for (int i = 0; i < 10; i++) {
+ measurements.add("s" + i);
+ }
+ measurementList.add(measurements);
+
+ List<List<TSDataType>> dataTypeList = new ArrayList<>();
+ dataTypeList.add(Collections.singletonList(TSDataType.INT64));
+ List<TSDataType> dataTypes = new ArrayList<>();
+ for (int i = 0; i < 10; i++) {
+ dataTypes.add(TSDataType.INT64);
+ }
+ dataTypeList.add(dataTypes);
+
+ List<List<TSEncoding>> encodingList = new ArrayList<>();
+ encodingList.add(Collections.singletonList(TSEncoding.RLE));
+ List<TSEncoding> encodings = new ArrayList<>();
+ for (int i = 0; i < 10; i++) {
+ encodings.add(TSEncoding.RLE);
+ }
+ encodingList.add(encodings);
+
+ List<CompressionType> compressionTypes = new ArrayList<>();
+ for (int i = 0; i < 11; i++) {
+ compressionTypes.add(CompressionType.SNAPPY);
+ }
+
+ session.createDeviceTemplate(
+ "template1", measurementList, dataTypeList, encodingList, compressionTypes);
+ }
}
diff --git a/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/NarrowConverter.scala b/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/NarrowConverter.scala
index 1291796..503f24f 100644
--- a/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/NarrowConverter.scala
+++ b/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/NarrowConverter.scala
@@ -36,7 +36,7 @@
import org.apache.iotdb.tsfile.read.filter.{TimeFilter, ValueFilter}
import org.apache.iotdb.tsfile.write.record.TSRecord
import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint
-import org.apache.iotdb.tsfile.write.schema.{MeasurementSchema, Schema}
+import org.apache.iotdb.tsfile.write.schema.{IMeasurementSchema, MeasurementSchema, Schema}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types._
@@ -527,7 +527,7 @@
* @param options encoding options
* @return MeasurementSchema
*/
- def getSeriesSchema(field: StructField, options: Map[String, String]): MeasurementSchema = {
+ def getSeriesSchema(field: StructField, options: Map[String, String]): IMeasurementSchema = {
val dataType = getTsDataType(field.dataType)
val encodingStr = dataType match {
case TSDataType.BOOLEAN => options.getOrElse(QueryConstant.BOOLEAN, TSEncoding.PLAIN.toString)
diff --git a/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/WideConverter.scala b/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/WideConverter.scala
index 293bb65..e7a8e42 100755
--- a/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/WideConverter.scala
+++ b/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/WideConverter.scala
@@ -35,10 +35,11 @@
import org.apache.iotdb.tsfile.utils.Binary
import org.apache.iotdb.tsfile.write.record.TSRecord
import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint
-import org.apache.iotdb.tsfile.write.schema.{MeasurementSchema, Schema}
+import org.apache.iotdb.tsfile.write.schema.{IMeasurementSchema, MeasurementSchema, Schema}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types._
+
import scala.collection.JavaConversions._
import scala.collection.mutable
import scala.collection.mutable.ListBuffer
@@ -421,7 +422,7 @@
* @param options encoding options
* @return MeasurementSchema
*/
- def getSeriesSchema(field: StructField, options: Map[String, String]): MeasurementSchema = {
+ def getSeriesSchema(field: StructField, options: Map[String, String]): IMeasurementSchema = {
val dataType = getTsDataType(field.dataType)
val encodingStr = dataType match {
case TSDataType.BOOLEAN => options.getOrElse(QueryConstant.BOOLEAN, TSEncoding.PLAIN.toString)
diff --git a/thrift/src/main/thrift/rpc.thrift b/thrift/src/main/thrift/rpc.thrift
index 94892fa..0b64889 100644
--- a/thrift/src/main/thrift/rpc.thrift
+++ b/thrift/src/main/thrift/rpc.thrift
@@ -271,6 +271,16 @@
9: optional string measurementAlias
}
+struct TSCreateAlignedTimeseriesReq {
+ 1: required i64 sessionId
+ 2: required string devicePath
+ 3: required list<string> measurements
+ 4: required list<i32> dataTypes
+ 5: required list<i32> encodings
+ 6: required i32 compressor
+ 7: optional list<string> measurementAlias
+}
+
struct TSRawDataQueryReq {
1: required i64 sessionId
2: required list<string> paths
@@ -299,6 +309,21 @@
3: required string timestampPrecision;
}
+struct TSSetDeviceTemplateReq {
+ 1: required i64 sessionId
+ 2: required string templateName
+ 3: required string prefixPath
+}
+
+struct TSCreateDeviceTemplateReq {
+ 1: required i64 sessionId
+ 2: required string name
+ 3: required list<list<string>> measurements
+ 4: required list<list<i32>> dataTypes
+ 5: required list<list<i32>> encodings
+ 6: required list<i32> compressors
+}
+
service TSIService {
TSOpenSessionResp openSession(1:TSOpenSessionReq req);
@@ -330,6 +355,8 @@
TSStatus createTimeseries(1:TSCreateTimeseriesReq req);
+ TSStatus createAlignedTimeseries(1:TSCreateAlignedTimeseriesReq req);
+
TSStatus createMultiTimeseries(1:TSCreateMultiTimeseriesReq req);
TSStatus deleteTimeseries(1:i64 sessionId, 2:list<string> path)
@@ -369,4 +396,8 @@
TSExecuteStatementResp executeRawDataQuery(1:TSRawDataQueryReq req);
i64 requestStatementId(1:i64 sessionId);
+
+ TSStatus createDeviceTemplate(1:TSCreateDeviceTemplateReq req);
+
+ TSStatus setDeviceTemplate(1:TSSetDeviceTemplateReq req);
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java
index fc70f03..d5ac0cb 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java
@@ -56,6 +56,7 @@
case INT32:
return new IntRleDecoder();
case INT64:
+ case VECTOR:
return new LongRleDecoder();
case FLOAT:
case DOUBLE:
@@ -68,6 +69,7 @@
case INT32:
return new DeltaBinaryDecoder.IntDeltaDecoder();
case INT64:
+ case VECTOR:
return new DeltaBinaryDecoder.LongDeltaDecoder();
case FLOAT:
case DOUBLE:
@@ -89,6 +91,7 @@
case INT32:
return new RegularDataDecoder.IntRegularDecoder();
case INT64:
+ case VECTOR:
return new RegularDataDecoder.LongRegularDecoder();
default:
throw new TsFileDecodingException(String.format(ERROR_MSG, encoding, dataType));
@@ -102,6 +105,7 @@
case INT32:
return new IntGorillaDecoder();
case INT64:
+ case VECTOR:
return new LongGorillaDecoder();
default:
throw new TsFileDecodingException(String.format(ERROR_MSG, encoding, dataType));
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkHeader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkHeader.java
index 663da60..22da502 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkHeader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkHeader.java
@@ -58,8 +58,21 @@
CompressionType compressionType,
TSEncoding encoding,
int numOfPages) {
+ this(measurementID, dataSize, dataType, compressionType, encoding, numOfPages, 0);
+ }
+
+ public ChunkHeader(
+ String measurementID,
+ int dataSize,
+ TSDataType dataType,
+ CompressionType compressionType,
+ TSEncoding encoding,
+ int numOfPages,
+ int mask) {
this(
- numOfPages <= 1 ? MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER : MetaMarker.CHUNK_HEADER,
+ (byte)
+ ((numOfPages <= 1 ? MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER : MetaMarker.CHUNK_HEADER)
+ | (byte) mask),
measurementID,
dataSize,
getSerializedSize(measurementID, dataSize),
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
index 07260bc..c0d30a2 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
@@ -34,7 +34,7 @@
import java.util.Objects;
/** Metadata of one chunk. */
-public class ChunkMetadata implements Accountable {
+public class ChunkMetadata implements Accountable, IChunkMetadata {
private String measurementUid;
@@ -71,6 +71,7 @@
private boolean isSeq = true;
private boolean isClosed;
private String filePath;
+ private byte mask;
private ChunkMetadata() {}
@@ -106,6 +107,7 @@
*
* @return Byte offset of header of this chunk (includes the marker)
*/
+ @Override
public long getOffsetOfChunkHeader() {
return offsetOfChunkHeader;
}
@@ -114,6 +116,7 @@
return measurementUid;
}
+ @Override
public Statistics getStatistics() {
return statistics;
}
@@ -161,7 +164,7 @@
chunkMetaData.offsetOfChunkHeader = ReadWriteIOUtils.readLong(buffer);
// if the TimeSeriesMetadataType is not 0, it means it has more than one chunk
// and each chunk's metadata has its own statistics
- if (timeseriesMetadata.getTimeSeriesMetadataType() != 0) {
+ if ((timeseriesMetadata.getTimeSeriesMetadataType() & 0x3F) != 0) {
chunkMetaData.statistics = Statistics.deserialize(buffer, chunkMetaData.tsDataType);
} else {
// if the TimeSeriesMetadataType is 0, it means it has only one chunk
@@ -171,10 +174,12 @@
return chunkMetaData;
}
+ @Override
public long getVersion() {
return version;
}
+ @Override
public void setVersion(long version) {
this.version = version;
}
@@ -239,10 +244,12 @@
measurementUid, deleteIntervalList, tsDataType, statistics, version, offsetOfChunkHeader);
}
+ @Override
public boolean isModified() {
return modified;
}
+ @Override
public void setModified(boolean modified) {
this.modified = modified;
}
@@ -283,10 +290,12 @@
this.ramSize = calculateRamSize();
}
+ @Override
public void setSeq(boolean seq) {
isSeq = seq;
}
+ @Override
public boolean isSeq() {
return isSeq;
}
@@ -306,4 +315,13 @@
public void setFilePath(String filePath) {
this.filePath = filePath;
}
+
+ @Override
+ public byte getMask() {
+ return mask;
+ }
+
+ public void setMask(byte mask) {
+ this.mask = mask;
+ }
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/IChunkMetadata.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/IChunkMetadata.java
new file mode 100644
index 0000000..b81a18b
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/IChunkMetadata.java
@@ -0,0 +1,74 @@
+/*
+ * 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.tsfile.file.metadata;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.controller.IChunkLoader;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.List;
+
+public interface IChunkMetadata {
+
+ Statistics getStatistics();
+
+ boolean isModified();
+
+ void setModified(boolean modified);
+
+ boolean isSeq();
+
+ void setSeq(boolean seq);
+
+ long getVersion();
+
+ void setVersion(long version);
+
+ long getOffsetOfChunkHeader();
+
+ long getStartTime();
+
+ long getEndTime();
+
+ boolean isFromOldTsFile();
+
+ IChunkLoader getChunkLoader();
+
+ void setChunkLoader(IChunkLoader chunkLoader);
+
+ void setFilePath(String filePath);
+
+ void setClosed(boolean closed);
+
+ TSDataType getDataType();
+
+ String getMeasurementUid();
+
+ void insertIntoSortedDeletions(long startTime, long endTime);
+
+ List<TimeRange> getDeleteIntervalList();
+
+ int serializeTo(OutputStream outputStream, boolean serializeStatistic) throws IOException;
+
+ byte getMask();
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ITimeSeriesMetadata.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ITimeSeriesMetadata.java
new file mode 100644
index 0000000..b508f57
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ITimeSeriesMetadata.java
@@ -0,0 +1,41 @@
+/*
+ * 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.tsfile.file.metadata;
+
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+
+import java.io.IOException;
+import java.util.List;
+
+public interface ITimeSeriesMetadata {
+
+ Statistics getStatistics();
+
+ boolean isModified();
+
+ void setModified(boolean modified);
+
+ boolean isSeq();
+
+ void setSeq(boolean seq);
+
+ List<IChunkMetadata> loadChunkMetadataList() throws IOException;
+
+ List<IChunkMetadata> getChunkMetadataList();
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java
index 7344c2e..df285dc 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java
@@ -33,14 +33,15 @@
import java.util.ArrayList;
import java.util.List;
-public class TimeseriesMetadata implements Accountable {
+public class TimeseriesMetadata implements Accountable, ITimeSeriesMetadata {
/** used for old version tsfile */
private long startOffsetOfChunkMetaDataList;
/**
* 0 means this time series has only one chunk, no need to save the statistic again in chunk
* metadata 1 means this time series has more than one chunk, should save the statistic again in
- * chunk metadata
+ * chunk metadata if the 8th bit is 1, it means it is the time column of a vector series if the
+ * 7th bit is 1, it means it is the value column of a vector series
*/
private byte timeSeriesMetadataType;
@@ -64,7 +65,7 @@
// used to save chunk metadata list while serializing
private PublicBAOS chunkMetadataListBuffer;
- private ArrayList<ChunkMetadata> chunkMetadataList;
+ private ArrayList<IChunkMetadata> chunkMetadataList;
public TimeseriesMetadata() {}
@@ -176,6 +177,7 @@
this.dataType = tsDataType;
}
+ @Override
public Statistics getStatistics() {
return statistics;
}
@@ -188,18 +190,25 @@
this.chunkMetadataLoader = chunkMetadataLoader;
}
- public List<ChunkMetadata> loadChunkMetadataList() throws IOException {
+ public IChunkMetadataLoader getChunkMetadataLoader() {
+ return chunkMetadataLoader;
+ }
+
+ @Override
+ public List<IChunkMetadata> loadChunkMetadataList() throws IOException {
return chunkMetadataLoader.loadChunkMetadataList(this);
}
- public List<ChunkMetadata> getChunkMetadataList() {
+ public List<IChunkMetadata> getChunkMetadataList() {
return chunkMetadataList;
}
+ @Override
public boolean isModified() {
return modified;
}
+ @Override
public void setModified(boolean modified) {
this.modified = modified;
}
@@ -214,10 +223,12 @@
return ramSize;
}
+ @Override
public void setSeq(boolean seq) {
isSeq = seq;
}
+ @Override
public boolean isSeq() {
return isSeq;
}
@@ -229,7 +240,7 @@
// For reading version-2 only
public void setChunkMetadataList(ArrayList<ChunkMetadata> chunkMetadataList) {
- this.chunkMetadataList = chunkMetadataList;
+ this.chunkMetadataList = new ArrayList<>(chunkMetadataList);
}
@Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/VectorChunkMetadata.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/VectorChunkMetadata.java
new file mode 100644
index 0000000..d576180
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/VectorChunkMetadata.java
@@ -0,0 +1,178 @@
+/*
+ * 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.tsfile.file.metadata;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.controller.IChunkLoader;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+public class VectorChunkMetadata implements IChunkMetadata {
+
+ private final IChunkMetadata timeChunkMetadata;
+ private final List<IChunkMetadata> valueChunkMetadataList;
+
+ public VectorChunkMetadata(
+ IChunkMetadata timeChunkMetadata, List<IChunkMetadata> valueChunkMetadataList) {
+ this.timeChunkMetadata = timeChunkMetadata;
+ this.valueChunkMetadataList = valueChunkMetadataList;
+ }
+
+ @Override
+ public Statistics getStatistics() {
+ return valueChunkMetadataList.size() == 1
+ ? valueChunkMetadataList.get(0).getStatistics()
+ : timeChunkMetadata.getStatistics();
+ }
+
+ @Override
+ public boolean isModified() {
+ return timeChunkMetadata.isModified();
+ }
+
+ @Override
+ public void setModified(boolean modified) {
+ timeChunkMetadata.setModified(modified);
+ }
+
+ @Override
+ public boolean isSeq() {
+ return timeChunkMetadata.isSeq();
+ }
+
+ @Override
+ public void setSeq(boolean seq) {
+ timeChunkMetadata.setSeq(seq);
+ }
+
+ @Override
+ public long getVersion() {
+ return timeChunkMetadata.getVersion();
+ }
+
+ @Override
+ public void setVersion(long version) {
+ timeChunkMetadata.setVersion(version);
+ }
+
+ @Override
+ public long getOffsetOfChunkHeader() {
+ return timeChunkMetadata.getOffsetOfChunkHeader();
+ }
+
+ @Override
+ public long getStartTime() {
+ return timeChunkMetadata.getStartTime();
+ }
+
+ @Override
+ public long getEndTime() {
+ return timeChunkMetadata.getEndTime();
+ }
+
+ @Override
+ public boolean isFromOldTsFile() {
+ return false;
+ }
+
+ @Override
+ public IChunkLoader getChunkLoader() {
+ return timeChunkMetadata.getChunkLoader();
+ }
+
+ @Override
+ public void setChunkLoader(IChunkLoader chunkLoader) {
+ timeChunkMetadata.setChunkLoader(chunkLoader);
+ for (IChunkMetadata chunkMetadata : valueChunkMetadataList) {
+ chunkMetadata.setChunkLoader(chunkLoader);
+ }
+ }
+
+ @Override
+ public void setFilePath(String filePath) {
+ timeChunkMetadata.setFilePath(filePath);
+ for (IChunkMetadata chunkMetadata : valueChunkMetadataList) {
+ chunkMetadata.setFilePath(filePath);
+ }
+ }
+
+ @Override
+ public void setClosed(boolean closed) {
+ timeChunkMetadata.setClosed(closed);
+ for (IChunkMetadata chunkMetadata : valueChunkMetadataList) {
+ chunkMetadata.setClosed(closed);
+ }
+ }
+
+ @Override
+ public TSDataType getDataType() {
+ return timeChunkMetadata.getDataType();
+ }
+
+ @Override
+ public String getMeasurementUid() {
+ return timeChunkMetadata.getMeasurementUid();
+ }
+
+ @Override
+ public void insertIntoSortedDeletions(long startTime, long endTime) {
+ timeChunkMetadata.insertIntoSortedDeletions(startTime, endTime);
+ }
+
+ @Override
+ public List<TimeRange> getDeleteIntervalList() {
+ return timeChunkMetadata.getDeleteIntervalList();
+ }
+
+ @Override
+ public int serializeTo(OutputStream outputStream, boolean serializeStatistic) {
+ throw new UnsupportedOperationException("VectorChunkMetadata doesn't support serial method");
+ }
+
+ @Override
+ public byte getMask() {
+ return 0;
+ }
+
+ public Chunk getTimeChunk() throws IOException {
+ return timeChunkMetadata.getChunkLoader().loadChunk((ChunkMetadata) timeChunkMetadata);
+ }
+
+ public List<Chunk> getValueChunkList() throws IOException {
+ List<Chunk> valueChunkList = new ArrayList<>();
+ for (IChunkMetadata chunkMetadata : valueChunkMetadataList) {
+ valueChunkList.add(chunkMetadata.getChunkLoader().loadChunk((ChunkMetadata) chunkMetadata));
+ }
+ return valueChunkList;
+ }
+
+ public IChunkMetadata getTimeChunkMetadata() {
+ return timeChunkMetadata;
+ }
+
+ public List<IChunkMetadata> getValueChunkMetadataList() {
+ return valueChunkMetadataList;
+ }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/VectorTimeSeriesMetadata.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/VectorTimeSeriesMetadata.java
new file mode 100644
index 0000000..67e6c0c
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/VectorTimeSeriesMetadata.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.tsfile.file.metadata;
+
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class VectorTimeSeriesMetadata implements ITimeSeriesMetadata {
+
+ private final TimeseriesMetadata timeseriesMetadata;
+ private final List<TimeseriesMetadata> valueTimeseriesMetadataList;
+
+ public VectorTimeSeriesMetadata(
+ TimeseriesMetadata timeseriesMetadata, List<TimeseriesMetadata> valueTimeseriesMetadataList) {
+ this.timeseriesMetadata = timeseriesMetadata;
+ this.valueTimeseriesMetadataList = valueTimeseriesMetadataList;
+ }
+
+ @Override
+ public Statistics getStatistics() {
+ return valueTimeseriesMetadataList.size() == 1
+ ? valueTimeseriesMetadataList.get(0).getStatistics()
+ : timeseriesMetadata.getStatistics();
+ }
+
+ @Override
+ public boolean isModified() {
+ return timeseriesMetadata.isModified();
+ }
+
+ @Override
+ public void setModified(boolean modified) {
+ timeseriesMetadata.setModified(modified);
+ }
+
+ @Override
+ public boolean isSeq() {
+ return timeseriesMetadata.isSeq();
+ }
+
+ @Override
+ public void setSeq(boolean seq) {
+ timeseriesMetadata.setSeq(seq);
+ }
+
+ @Override
+ public List<IChunkMetadata> loadChunkMetadataList() throws IOException {
+ if (timeseriesMetadata.getChunkMetadataLoader().isMemChunkMetadataLoader()) {
+ return timeseriesMetadata.loadChunkMetadataList();
+ } else {
+ List<IChunkMetadata> timeChunkMetadata = timeseriesMetadata.loadChunkMetadataList();
+ List<List<IChunkMetadata>> valueChunkMetadataList = new ArrayList<>();
+ for (TimeseriesMetadata metadata : valueTimeseriesMetadataList) {
+ valueChunkMetadataList.add(metadata.loadChunkMetadataList());
+ }
+
+ List<IChunkMetadata> res = new ArrayList<>();
+
+ for (int i = 0; i < timeChunkMetadata.size(); i++) {
+ List<IChunkMetadata> chunkMetadataList = new ArrayList<>();
+ for (List<IChunkMetadata> chunkMetadata : valueChunkMetadataList) {
+ chunkMetadataList.add(chunkMetadata.get(i));
+ }
+ res.add(new VectorChunkMetadata(timeChunkMetadata.get(i), chunkMetadataList));
+ }
+ return res;
+ }
+ }
+
+ @Override
+ public List<IChunkMetadata> getChunkMetadataList() {
+ return null;
+ }
+
+ public List<TimeseriesMetadata> getValueTimeseriesMetadataList() {
+ return valueTimeseriesMetadataList;
+ }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSDataType.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSDataType.java
index 790462a..7c1c9d7 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSDataType.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSDataType.java
@@ -41,7 +41,10 @@
DOUBLE((byte) 4),
/** TEXT */
- TEXT((byte) 5);
+ TEXT((byte) 5),
+
+ /** VECTOR */
+ VECTOR((byte) 6);
private final byte type;
@@ -73,6 +76,8 @@
return TSDataType.DOUBLE;
case 5:
return TSDataType.TEXT;
+ case 6:
+ return TSDataType.VECTOR;
default:
throw new IllegalArgumentException("Invalid input: " + type);
}
@@ -105,6 +110,7 @@
case TEXT:
case INT64:
case DOUBLE:
+ case VECTOR:
return 8;
default:
throw new UnSupportedDataTypeException(this.toString());
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/Statistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/Statistics.java
index 1ec2001..cd1abe1 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/Statistics.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/Statistics.java
@@ -77,6 +77,8 @@
return new DoubleStatistics();
case FLOAT:
return new FloatStatistics();
+ case VECTOR:
+ return new TimeStatistics();
default:
throw new UnknownColumnTypeException(type.toString());
}
@@ -96,6 +98,8 @@
return DoubleStatistics.DOUBLE_STATISTICS_FIXED_RAM_SIZE;
case FLOAT:
return FloatStatistics.FLOAT_STATISTICS_FIXED_RAM_SIZE;
+ case VECTOR:
+ return TimeStatistics.TIME_STATISTICS_FIXED_RAM_SIZE;
default:
throw new UnknownColumnTypeException(type.toString());
}
@@ -189,61 +193,36 @@
}
public void update(long time, boolean value) {
- if (time < this.startTime) {
- startTime = time;
- }
- if (time > this.endTime) {
- endTime = time;
- }
- count++;
+ update(time);
updateStats(value);
}
public void update(long time, int value) {
- if (time < this.startTime) {
- startTime = time;
- }
- if (time > this.endTime) {
- endTime = time;
- }
- count++;
+ update(time);
updateStats(value);
}
public void update(long time, long value) {
- if (time < this.startTime) {
- startTime = time;
- }
- if (time > this.endTime) {
- endTime = time;
- }
- count++;
+ update(time);
updateStats(value);
}
public void update(long time, float value) {
- if (time < this.startTime) {
- startTime = time;
- }
- if (time > this.endTime) {
- endTime = time;
- }
- count++;
+ update(time);
updateStats(value);
}
public void update(long time, double value) {
- if (time < this.startTime) {
- startTime = time;
- }
- if (time > this.endTime) {
- endTime = time;
- }
- count++;
+ update(time);
updateStats(value);
}
public void update(long time, Binary value) {
+ update(time);
+ updateStats(value);
+ }
+
+ public void update(long time) {
if (time < startTime) {
startTime = time;
}
@@ -251,65 +230,39 @@
endTime = time;
}
count++;
- updateStats(value);
}
public void update(long[] time, boolean[] values, int batchSize) {
- if (time[0] < startTime) {
- startTime = time[0];
- }
- if (time[batchSize - 1] > this.endTime) {
- endTime = time[batchSize - 1];
- }
- count += batchSize;
+ update(time, batchSize);
updateStats(values, batchSize);
}
public void update(long[] time, int[] values, int batchSize) {
- if (time[0] < startTime) {
- startTime = time[0];
- }
- if (time[batchSize - 1] > this.endTime) {
- endTime = time[batchSize - 1];
- }
- count += batchSize;
+ update(time, batchSize);
updateStats(values, batchSize);
}
public void update(long[] time, long[] values, int batchSize) {
- if (time[0] < startTime) {
- startTime = time[0];
- }
- if (time[batchSize - 1] > this.endTime) {
- endTime = time[batchSize - 1];
- }
- count += batchSize;
+ update(time, batchSize);
updateStats(values, batchSize);
}
public void update(long[] time, float[] values, int batchSize) {
- if (time[0] < startTime) {
- startTime = time[0];
- }
- if (time[batchSize - 1] > this.endTime) {
- endTime = time[batchSize - 1];
- }
- count += batchSize;
+ update(time, batchSize);
updateStats(values, batchSize);
}
public void update(long[] time, double[] values, int batchSize) {
- if (time[0] < startTime) {
- startTime = time[0];
- }
- if (time[batchSize - 1] > this.endTime) {
- endTime = time[batchSize - 1];
- }
- count += batchSize;
+ update(time, batchSize);
updateStats(values, batchSize);
}
public void update(long[] time, Binary[] values, int batchSize) {
+ update(time, batchSize);
+ updateStats(values, batchSize);
+ }
+
+ public void update(long[] time, int batchSize) {
if (time[0] < startTime) {
startTime = time[0];
}
@@ -317,7 +270,6 @@
endTime = time[batchSize - 1];
}
count += batchSize;
- updateStats(values, batchSize);
}
protected abstract void mergeStatisticsValue(Statistics stats);
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/TimeStatistics.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/TimeStatistics.java
new file mode 100644
index 0000000..e812166
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/statistics/TimeStatistics.java
@@ -0,0 +1,161 @@
+/*
+ * 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.tsfile.file.metadata.statistics;
+
+import org.apache.iotdb.tsfile.exception.filter.StatisticsClassException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+public class TimeStatistics extends Statistics {
+
+ static final int TIME_STATISTICS_FIXED_RAM_SIZE = 40;
+
+ @Override
+ public TSDataType getType() {
+ return TSDataType.VECTOR;
+ }
+
+ @Override
+ public int getStatsSize() {
+ return 0;
+ }
+
+ @Override
+ public void setMinMaxFromBytes(byte[] minBytes, byte[] maxBytes) {
+ throw new StatisticsClassException("Time statistics does not support: set min max from bytes");
+ }
+
+ @Override
+ public Long getMinValue() {
+ throw new StatisticsClassException("Time statistics does not support: min value");
+ }
+
+ @Override
+ public Long getMaxValue() {
+ throw new StatisticsClassException("Time statistics does not support: max value");
+ }
+
+ @Override
+ public Long getFirstValue() {
+ throw new StatisticsClassException("Time statistics does not support: first value");
+ }
+
+ @Override
+ public Long getLastValue() {
+ throw new StatisticsClassException("Time statistics does not support: last value");
+ }
+
+ @Override
+ public double getSumDoubleValue() {
+ throw new StatisticsClassException("Time statistics does not support: double sum");
+ }
+
+ @Override
+ public long getSumLongValue() {
+ throw new StatisticsClassException("Time statistics does not support: long sum");
+ }
+
+ @Override
+ void updateStats(long value) {
+ throw new StatisticsClassException("Time statistics does not support: update stats");
+ }
+
+ @Override
+ void updateStats(long[] values, int batchSize) {
+ throw new StatisticsClassException("Time statistics does not support: update stats");
+ }
+
+ @Override
+ public void updateStats(long minValue, long maxValue) {
+ throw new StatisticsClassException("Time statistics does not support: update stats");
+ }
+
+ @Override
+ public long calculateRamSize() {
+ return TIME_STATISTICS_FIXED_RAM_SIZE;
+ }
+
+ @Override
+ protected void mergeStatisticsValue(Statistics stats) {}
+
+ @Override
+ public byte[] getMinValueBytes() {
+ throw new StatisticsClassException("Time statistics does not support: get min value bytes");
+ }
+
+ @Override
+ public byte[] getMaxValueBytes() {
+ throw new StatisticsClassException("Time statistics does not support: get max value bytes");
+ }
+
+ @Override
+ public byte[] getFirstValueBytes() {
+ throw new StatisticsClassException("Time statistics does not support: get first value bytes");
+ }
+
+ @Override
+ public byte[] getLastValueBytes() {
+ throw new StatisticsClassException("Time statistics does not support: get last value bytes");
+ }
+
+ @Override
+ public byte[] getSumValueBytes() {
+ throw new StatisticsClassException("Time statistics does not support: get sum value bytes");
+ }
+
+ @Override
+ public ByteBuffer getMinValueBuffer() {
+ throw new StatisticsClassException("Time statistics does not support: get min value bytes");
+ }
+
+ @Override
+ public ByteBuffer getMaxValueBuffer() {
+ throw new StatisticsClassException("Time statistics does not support: get max value buffer");
+ }
+
+ @Override
+ public ByteBuffer getFirstValueBuffer() {
+ throw new StatisticsClassException("Time statistics does not support: get first value buffer");
+ }
+
+ @Override
+ public ByteBuffer getLastValueBuffer() {
+ throw new StatisticsClassException("Time statistics does not support: get last value buffer");
+ }
+
+ @Override
+ public ByteBuffer getSumValueBuffer() {
+ throw new StatisticsClassException("Time statistics does not support: get sum value buffer");
+ }
+
+ @Override
+ public int serializeStats(OutputStream outputStream) {
+ return 0;
+ }
+
+ @Override
+ public void deserialize(InputStream inputStream) throws IOException {}
+
+ @Override
+ public void deserialize(ByteBuffer byteBuffer) {}
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
index bc63d93..41d0cee 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
@@ -29,6 +29,7 @@
import org.apache.iotdb.tsfile.file.header.PageHeader;
import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetadata;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.file.metadata.MetadataIndexEntry;
import org.apache.iotdb.tsfile.file.metadata.MetadataIndexNode;
import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
@@ -48,6 +49,7 @@
import org.apache.iotdb.tsfile.utils.BloomFilter;
import org.apache.iotdb.tsfile.utils.Pair;
import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.slf4j.Logger;
@@ -550,7 +552,10 @@
Map<String, List<ChunkMetadata>> seriesMetadata = new HashMap<>();
for (TimeseriesMetadata timeseriesMetadata : timeseriesMetadataMap) {
seriesMetadata.put(
- timeseriesMetadata.getMeasurementId(), timeseriesMetadata.getChunkMetadataList());
+ timeseriesMetadata.getMeasurementId(),
+ timeseriesMetadata.getChunkMetadataList().stream()
+ .map(chunkMetadata -> ((ChunkMetadata) chunkMetadata))
+ .collect(Collectors.toList()));
}
return seriesMetadata;
}
@@ -945,7 +950,7 @@
*/
@SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
public long selfCheck(
- Map<Path, MeasurementSchema> newSchema,
+ Map<Path, IMeasurementSchema> newSchema,
List<ChunkGroupMetadata> chunkGroupMetadataList,
boolean fastFinish)
throws IOException {
@@ -986,19 +991,21 @@
long truncatedSize = headerLength;
byte marker;
String lastDeviceId = null;
- List<MeasurementSchema> measurementSchemaList = new ArrayList<>();
+ List<IMeasurementSchema> measurementSchemaList = new ArrayList<>();
try {
while ((marker = this.readMarker()) != MetaMarker.SEPARATOR) {
switch (marker) {
case MetaMarker.CHUNK_HEADER:
case MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER:
+ case (byte) (MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER | 0x80):
+ case (byte) (MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER | 0x40):
fileOffsetOfChunk = this.position() - 1;
// if there is something wrong with a chunk, we will drop the whole ChunkGroup
// as different chunks may be created by the same insertions(sqls), and partial
// insertion is not tolerable
ChunkHeader chunkHeader = this.readChunkHeader(marker);
measurementID = chunkHeader.getMeasurementID();
- MeasurementSchema measurementSchema =
+ IMeasurementSchema measurementSchema =
new MeasurementSchema(
measurementID,
chunkHeader.getDataType(),
@@ -1008,7 +1015,7 @@
dataType = chunkHeader.getDataType();
Statistics<?> chunkStatistics = Statistics.getStatsByType(dataType);
int dataSize = chunkHeader.getDataSize();
- if (chunkHeader.getChunkType() == MetaMarker.CHUNK_HEADER) {
+ if (((byte) (chunkHeader.getChunkType() & 0x3F)) == MetaMarker.CHUNK_HEADER) {
while (dataSize > 0) {
// a new Page
PageHeader pageHeader = this.readPageHeader(chunkHeader.getDataType(), true);
@@ -1077,7 +1084,7 @@
if (lastDeviceId != null) {
// schema of last chunk group
if (newSchema != null) {
- for (MeasurementSchema tsSchema : measurementSchemaList) {
+ for (IMeasurementSchema tsSchema : measurementSchemaList) {
newSchema.putIfAbsent(
new Path(lastDeviceId, tsSchema.getMeasurementId()), tsSchema);
}
@@ -1096,7 +1103,7 @@
if (lastDeviceId != null) {
// schema of last chunk group
if (newSchema != null) {
- for (MeasurementSchema tsSchema : measurementSchemaList) {
+ for (IMeasurementSchema tsSchema : measurementSchemaList) {
newSchema.putIfAbsent(
new Path(lastDeviceId, tsSchema.getMeasurementId()), tsSchema);
}
@@ -1119,7 +1126,7 @@
if (lastDeviceId != null) {
// schema of last chunk group
if (newSchema != null) {
- for (MeasurementSchema tsSchema : measurementSchemaList) {
+ for (IMeasurementSchema tsSchema : measurementSchemaList) {
newSchema.putIfAbsent(new Path(lastDeviceId, tsSchema.getMeasurementId()), tsSchema);
}
}
@@ -1151,7 +1158,7 @@
return Collections.emptyList();
}
List<ChunkMetadata> chunkMetadataList = readChunkMetaDataList(timeseriesMetaData);
- chunkMetadataList.sort(Comparator.comparingLong(ChunkMetadata::getStartTime));
+ chunkMetadataList.sort(Comparator.comparingLong(IChunkMetadata::getStartTime));
return chunkMetadataList;
}
@@ -1162,7 +1169,9 @@
*/
public List<ChunkMetadata> readChunkMetaDataList(TimeseriesMetadata timeseriesMetaData)
throws IOException {
- return timeseriesMetaData.getChunkMetadataList();
+ return timeseriesMetaData.getChunkMetadataList().stream()
+ .map(chunkMetadata -> (ChunkMetadata) chunkMetadata)
+ .collect(Collectors.toList());
}
/**
@@ -1311,9 +1320,12 @@
timeseriesMetadataList.add(TimeseriesMetadata.deserializeFrom(nextBuffer, true));
}
for (TimeseriesMetadata timeseriesMetadata : timeseriesMetadataList) {
- measurementChunkMetadataList
- .computeIfAbsent(timeseriesMetadata.getMeasurementId(), m -> new ArrayList<>())
- .addAll(timeseriesMetadata.getChunkMetadataList());
+ List<ChunkMetadata> list =
+ measurementChunkMetadataList.computeIfAbsent(
+ timeseriesMetadata.getMeasurementId(), m -> new ArrayList<>());
+ for (IChunkMetadata chunkMetadata : timeseriesMetadata.getChunkMetadataList()) {
+ list.add((ChunkMetadata) chunkMetadata);
+ }
}
return measurementChunkMetadataList;
} catch (IOException e) {
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/BatchData.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/BatchData.java
index c2a7d3b..9080ba4 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/BatchData.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/BatchData.java
@@ -31,6 +31,7 @@
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 org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsVector;
import java.util.ArrayList;
import java.util.List;
@@ -77,6 +78,7 @@
protected List<float[]> floatRet;
protected List<double[]> doubleRet;
protected List<Binary[]> binaryRet;
+ protected List<TsPrimitiveType[][]> vectorRet;
public BatchData() {
dataType = null;
@@ -134,6 +136,8 @@
return getBoolean();
case TEXT:
return getBinary();
+ case VECTOR:
+ return getVector();
default:
return null;
}
@@ -153,6 +157,8 @@
return new TsBoolean(getBoolean());
case TEXT:
return new TsBinary(getBinary());
+ case VECTOR:
+ return new TsVector(getVector());
default:
return null;
}
@@ -203,6 +209,10 @@
binaryRet = new ArrayList<>();
binaryRet.add(new Binary[capacity]);
break;
+ case VECTOR:
+ vectorRet = new ArrayList<>();
+ vectorRet.add(new TsPrimitiveType[capacity][]);
+ break;
default:
throw new UnSupportedDataTypeException(String.valueOf(dataType));
}
@@ -417,6 +427,41 @@
count++;
}
+ /**
+ * put vector data.
+ *
+ * @param t timestamp
+ * @param v vector data.
+ */
+ public void putVector(long t, TsPrimitiveType[] v) {
+ if (writeCurArrayIndex == capacity) {
+ if (capacity >= CAPACITY_THRESHOLD) {
+ timeRet.add(new long[capacity]);
+ vectorRet.add(new TsPrimitiveType[capacity][]);
+ writeCurListIndex++;
+ writeCurArrayIndex = 0;
+ } else {
+ int newCapacity = capacity << 1;
+
+ long[] newTimeData = new long[newCapacity];
+ TsPrimitiveType[][] newValueData = new TsPrimitiveType[newCapacity][];
+
+ System.arraycopy(timeRet.get(0), 0, newTimeData, 0, capacity);
+ System.arraycopy(vectorRet.get(0), 0, newValueData, 0, capacity);
+
+ timeRet.set(0, newTimeData);
+ vectorRet.set(0, newValueData);
+
+ capacity = newCapacity;
+ }
+ }
+ timeRet.get(writeCurListIndex)[writeCurArrayIndex] = t;
+ vectorRet.get(writeCurListIndex)[writeCurArrayIndex] = v;
+
+ writeCurArrayIndex++;
+ count++;
+ }
+
public boolean getBoolean() {
return this.booleanRet.get(readCurListIndex)[readCurArrayIndex];
}
@@ -465,6 +510,14 @@
this.binaryRet.get(readCurListIndex)[readCurArrayIndex] = v;
}
+ public TsPrimitiveType[] getVector() {
+ return this.vectorRet.get(readCurListIndex)[readCurArrayIndex];
+ }
+
+ public void setVector(TsPrimitiveType[] v) {
+ this.vectorRet.get(readCurListIndex)[readCurArrayIndex] = v;
+ }
+
public void setTime(long v) {
this.timeRet.get(readCurListIndex)[readCurArrayIndex] = v;
}
@@ -495,6 +548,9 @@
case TEXT:
putBinary(t, (Binary) v);
break;
+ case VECTOR:
+ putVector(t, (TsPrimitiveType[]) v);
+ break;
default:
throw new UnSupportedDataTypeException(String.valueOf(dataType));
}
@@ -532,6 +588,10 @@
return booleanRet.get(idx / capacity)[idx % capacity];
}
+ public TsPrimitiveType[] getVectorByIndex(int idx) {
+ return vectorRet.get(idx / capacity)[idx % capacity];
+ }
+
public TimeValuePair getLastPairBeforeOrEqualTimestamp(long queryTime) {
TimeValuePair resultPair = new TimeValuePair(Long.MIN_VALUE, null);
resetBatchData();
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Chunk.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Chunk.java
index a6ba83d..90920ab 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Chunk.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Chunk.java
@@ -76,7 +76,8 @@
// if the merged chunk has only one page, after merge with current chunk ,it will have more than
// page
// so we should add page statistics for it
- if (chunk.chunkHeader.getChunkType() == MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER) {
+ if (((byte) (chunk.chunkHeader.getChunkType() & 0x3F))
+ == MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER) {
// read the uncompressedSize and compressedSize of this page
ReadWriteForEncodingUtils.readUnsignedVarInt(chunk.chunkData);
ReadWriteForEncodingUtils.readUnsignedVarInt(chunk.chunkData);
@@ -96,7 +97,7 @@
// if the current chunk has only one page, after merge with the merged chunk ,it will have more
// than page
// so we should add page statistics for it
- if (chunkHeader.getChunkType() == MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER) {
+ if (((byte) (chunkHeader.getChunkType() & 0x3F)) == MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER) {
// change the chunk type
chunkHeader.setChunkType(MetaMarker.CHUNK_HEADER);
// read the uncompressedSize and compressedSize of this page
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/IChunkMetadataLoader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/IChunkMetadataLoader.java
index f02f9e3..71371a4 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/IChunkMetadataLoader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/IChunkMetadataLoader.java
@@ -18,8 +18,8 @@
*/
package org.apache.iotdb.tsfile.read.controller;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
-import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.ITimeSeriesMetadata;
import java.io.IOException;
import java.util.List;
@@ -27,6 +27,8 @@
public interface IChunkMetadataLoader {
/** read all chunk metadata of one time series in one file. */
- List<ChunkMetadata> loadChunkMetadataList(TimeseriesMetadata timeseriesMetadata)
+ List<IChunkMetadata> loadChunkMetadataList(ITimeSeriesMetadata timeseriesMetadata)
throws IOException;
+
+ boolean isMemChunkMetadataLoader();
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/IMetadataQuerier.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/IMetadataQuerier.java
index 704aa11..77eb09d 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/IMetadataQuerier.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/IMetadataQuerier.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.tsfile.read.controller;
import org.apache.iotdb.tsfile.exception.write.NoMeasurementException;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.file.metadata.TsFileMetadata;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.common.Path;
@@ -31,9 +31,9 @@
public interface IMetadataQuerier {
- List<ChunkMetadata> getChunkMetaDataList(Path path) throws IOException;
+ List<IChunkMetadata> getChunkMetaDataList(Path path) throws IOException;
- Map<Path, List<ChunkMetadata>> getChunkMetaDataMap(List<Path> paths) throws IOException;
+ Map<Path, List<IChunkMetadata>> getChunkMetaDataMap(List<Path> paths) throws IOException;
TsFileMetadata getWholeFileMetadata();
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/MetadataQuerierByFileImpl.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/MetadataQuerierByFileImpl.java
index dbe7f7b..98d28b9 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/MetadataQuerierByFileImpl.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/MetadataQuerierByFileImpl.java
@@ -20,6 +20,7 @@
import org.apache.iotdb.tsfile.common.cache.LRUCache;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
import org.apache.iotdb.tsfile.file.metadata.TsFileMetadata;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -65,13 +66,13 @@
}
@Override
- public List<ChunkMetadata> getChunkMetaDataList(Path path) throws IOException {
- return chunkMetaDataCache.get(path);
+ public List<IChunkMetadata> getChunkMetaDataList(Path path) throws IOException {
+ return new ArrayList<>(chunkMetaDataCache.get(path));
}
@Override
- public Map<Path, List<ChunkMetadata>> getChunkMetaDataMap(List<Path> paths) throws IOException {
- Map<Path, List<ChunkMetadata>> chunkMetaDatas = new HashMap<>();
+ public Map<Path, List<IChunkMetadata>> getChunkMetaDataMap(List<Path> paths) throws IOException {
+ Map<Path, List<IChunkMetadata>> chunkMetaDatas = new HashMap<>();
for (Path path : paths) {
if (!chunkMetaDatas.containsKey(path)) {
chunkMetaDatas.put(path, new ArrayList<>());
@@ -202,7 +203,7 @@
continue;
}
- for (ChunkMetadata chunkMetadata : seriesMetadata.getValue()) {
+ for (IChunkMetadata chunkMetadata : seriesMetadata.getValue()) {
LocateStatus location =
checkLocateStatus(chunkMetadata, spacePartitionStartPos, spacePartitionEndPos);
if (location == LocateStatus.after) {
@@ -250,7 +251,7 @@
* @return LocateStatus
*/
public static LocateStatus checkLocateStatus(
- ChunkMetadata chunkMetaData, long spacePartitionStartPos, long spacePartitionEndPos) {
+ IChunkMetadata chunkMetaData, long spacePartitionStartPos, long spacePartitionEndPos) {
long startOffsetOfChunk = chunkMetaData.getOffsetOfChunkHeader();
if (spacePartitionStartPos <= startOffsetOfChunk && startOffsetOfChunk < spacePartitionEndPos) {
return LocateStatus.in;
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/SingleSeriesExpression.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/SingleSeriesExpression.java
index 6cd072e..f7b603b 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/SingleSeriesExpression.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/SingleSeriesExpression.java
@@ -65,4 +65,8 @@
public Path getSeriesPath() {
return this.seriesPath;
}
+
+ public void setSeriesPath(Path seriesPath) {
+ this.seriesPath = seriesPath;
+ }
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/ValueFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/ValueFilter.java
index 4420db9..036cfed 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/ValueFilter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/ValueFilter.java
@@ -28,6 +28,7 @@
import org.apache.iotdb.tsfile.read.filter.operator.LtEq;
import org.apache.iotdb.tsfile.read.filter.operator.NotEq;
import org.apache.iotdb.tsfile.read.filter.operator.NotFilter;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
import java.util.Set;
@@ -74,6 +75,21 @@
}
}
+ public static class VectorValueIn<T extends Comparable<T>> extends ValueIn<T> {
+
+ private final int index;
+
+ private VectorValueIn(Set<T> values, boolean not, int index) {
+ super(values, not);
+ this.index = index;
+ }
+
+ public boolean satisfy(long time, TsPrimitiveType[] values) {
+ Object v = filterType == FilterType.TIME_FILTER ? time : values[index].getValue();
+ return this.values.contains(v) != not;
+ }
+ }
+
public static class ValueEq<T extends Comparable<T>> extends Eq<T> {
private ValueEq(T value) {
@@ -81,6 +97,21 @@
}
}
+ public static class VectorValueEq<T extends Comparable<T>> extends ValueEq<T> {
+
+ private final int index;
+
+ private VectorValueEq(T value, int index) {
+ super(value);
+ this.index = index;
+ }
+
+ public boolean satisfy(long time, TsPrimitiveType[] values) {
+ Object v = filterType == FilterType.TIME_FILTER ? time : values[index].getValue();
+ return this.value.equals(v);
+ }
+ }
+
public static class ValueGt<T extends Comparable<T>> extends Gt<T> {
private ValueGt(T value) {
@@ -88,6 +119,21 @@
}
}
+ public static class VectorValueGt<T extends Comparable<T>> extends ValueGt<T> {
+
+ private final int index;
+
+ private VectorValueGt(T value, int index) {
+ super(value);
+ this.index = index;
+ }
+
+ public boolean satisfy(long time, TsPrimitiveType[] values) {
+ Object v = filterType == FilterType.TIME_FILTER ? time : values[index].getValue();
+ return this.value.compareTo((T) v) < 0;
+ }
+ }
+
public static class ValueGtEq<T extends Comparable<T>> extends GtEq<T> {
private ValueGtEq(T value) {
@@ -95,6 +141,21 @@
}
}
+ public static class VectorValueGtEq<T extends Comparable<T>> extends ValueGtEq<T> {
+
+ private final int index;
+
+ private VectorValueGtEq(T value, int index) {
+ super(value);
+ this.index = index;
+ }
+
+ public boolean satisfy(long time, TsPrimitiveType[] values) {
+ Object v = filterType == FilterType.TIME_FILTER ? time : values[index].getValue();
+ return this.value.compareTo((T) v) <= 0;
+ }
+ }
+
public static class ValueLt<T extends Comparable<T>> extends Lt<T> {
private ValueLt(T value) {
@@ -102,6 +163,21 @@
}
}
+ public static class VectorValueLt<T extends Comparable<T>> extends ValueLt<T> {
+
+ private final int index;
+
+ private VectorValueLt(T value, int index) {
+ super(value);
+ this.index = index;
+ }
+
+ public boolean satisfy(long time, TsPrimitiveType[] values) {
+ Object v = filterType == FilterType.TIME_FILTER ? time : values[index].getValue();
+ return this.value.compareTo((T) v) > 0;
+ }
+ }
+
public static class ValueLtEq<T extends Comparable<T>> extends LtEq<T> {
private ValueLtEq(T value) {
@@ -109,6 +185,21 @@
}
}
+ public static class VectorValueLtEq<T extends Comparable<T>> extends ValueLtEq<T> {
+
+ private final int index;
+
+ private VectorValueLtEq(T value, int index) {
+ super(value);
+ this.index = index;
+ }
+
+ public boolean satisfy(long time, TsPrimitiveType[] values) {
+ Object v = filterType == FilterType.TIME_FILTER ? time : values[index].getValue();
+ return this.value.compareTo((T) v) >= 0;
+ }
+ }
+
public static class ValueNotFilter extends NotFilter {
private ValueNotFilter(Filter filter) {
@@ -127,4 +218,19 @@
super(value, FilterType.VALUE_FILTER);
}
}
+
+ public static class VectorValueNotEq<T extends Comparable<T>> extends ValueNotEq<T> {
+
+ private final int index;
+
+ private VectorValueNotEq(T value, int index) {
+ super(value);
+ this.index = index;
+ }
+
+ public boolean satisfy(long time, TsPrimitiveType[] values) {
+ Object v = filterType == FilterType.TIME_FILTER ? time : values[index].getValue();
+ return !this.value.equals(v);
+ }
+ }
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/In.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/In.java
index 0c70f13..c76154a 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/In.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/In.java
@@ -42,11 +42,11 @@
private static final long serialVersionUID = 8572705136773595399L;
- private Set<T> values;
+ protected Set<T> values;
- private boolean not;
+ protected boolean not;
- private FilterType filterType;
+ protected FilterType filterType;
public In() {}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/ExecutorWithTimeGenerator.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/ExecutorWithTimeGenerator.java
index 90e41d4..bfb8ea5 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/ExecutorWithTimeGenerator.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/ExecutorWithTimeGenerator.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.tsfile.read.query.executor;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.common.Path;
import org.apache.iotdb.tsfile.read.controller.IChunkLoader;
@@ -76,7 +76,7 @@
boolean cachedValue = cachedIterator.next();
Path selectedPath = selectedPathIterator.next();
- List<ChunkMetadata> chunkMetadataList = metadataQuerier.getChunkMetaDataList(selectedPath);
+ List<IChunkMetadata> chunkMetadataList = metadataQuerier.getChunkMetaDataList(selectedPath);
if (chunkMetadataList.size() != 0) {
dataTypes.add(chunkMetadataList.get(0).getDataType());
if (cachedValue) {
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/TsFileExecutor.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/TsFileExecutor.java
index 788d541..a530cb3 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/TsFileExecutor.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/TsFileExecutor.java
@@ -20,7 +20,7 @@
import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
import org.apache.iotdb.tsfile.exception.write.NoMeasurementException;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.common.Path;
import org.apache.iotdb.tsfile.read.common.TimeRange;
@@ -176,7 +176,7 @@
List<TSDataType> dataTypes = new ArrayList<>();
for (Path path : selectedPathList) {
- List<ChunkMetadata> chunkMetadataList = metadataQuerier.getChunkMetaDataList(path);
+ List<IChunkMetadata> chunkMetadataList = metadataQuerier.getChunkMetaDataList(path);
AbstractFileSeriesReader seriesReader;
if (chunkMetadataList.isEmpty()) {
seriesReader = new EmptyFileSeriesReader();
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/TsFileTimeGenerator.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/TsFileTimeGenerator.java
index c68fc74..571f70a 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/TsFileTimeGenerator.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/TsFileTimeGenerator.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.tsfile.read.query.timegenerator;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.read.controller.IChunkLoader;
import org.apache.iotdb.tsfile.read.controller.IMetadataQuerier;
import org.apache.iotdb.tsfile.read.expression.IExpression;
@@ -46,7 +46,7 @@
@Override
protected IBatchReader generateNewBatchReader(SingleSeriesExpression expression)
throws IOException {
- List<ChunkMetadata> chunkMetadataList =
+ List<IChunkMetadata> chunkMetadataList =
metadataQuerier.getChunkMetaDataList(expression.getSeriesPath());
return new FileSeriesReader(chunkLoader, chunkMetadataList, expression.getFilter());
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java
index 2a4e1f7..30ba34d 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java
@@ -84,7 +84,7 @@
while (chunkDataBuffer.remaining() > 0) {
// deserialize a PageHeader from chunkDataBuffer
PageHeader pageHeader;
- if (chunkHeader.getChunkType() == MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER) {
+ if (((byte) (chunkHeader.getChunkType() & 0x3F)) == MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER) {
pageHeader = PageHeader.deserializeFrom(chunkDataBuffer, chunkStatistic);
} else {
pageHeader = PageHeader.deserializeFrom(chunkDataBuffer, chunkHeader.getDataType());
@@ -118,11 +118,11 @@
return pageReaderList.remove(0).getAllSatisfiedPageData();
}
- private void skipBytesInStreamByLength(long length) {
- chunkDataBuffer.position(chunkDataBuffer.position() + (int) length);
+ private void skipBytesInStreamByLength(int length) {
+ chunkDataBuffer.position(chunkDataBuffer.position() + length);
}
- public boolean pageSatisfied(PageHeader pageHeader) {
+ protected boolean pageSatisfied(PageHeader pageHeader) {
if (deleteIntervalList != null) {
for (TimeRange range : deleteIntervalList) {
if (range.contains(pageHeader.getStartTime(), pageHeader.getEndTime())) {
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/VectorChunkReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/VectorChunkReader.java
new file mode 100644
index 0000000..37afc31
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/VectorChunkReader.java
@@ -0,0 +1,262 @@
+/*
+ * 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.tsfile.read.reader.chunk;
+
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.compress.IUnCompressor;
+import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.read.reader.IChunkReader;
+import org.apache.iotdb.tsfile.read.reader.IPageReader;
+import org.apache.iotdb.tsfile.read.reader.page.VectorPageReader;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+public class VectorChunkReader implements IChunkReader {
+
+ private final ChunkHeader timeChunkHeader;
+ private final List<ChunkHeader> valueChunkHeaderList = new ArrayList<>();
+ private final ByteBuffer timeChunkDataBuffer;
+ private final List<ByteBuffer> valueChunkDataBufferList = new ArrayList<>();
+ private final IUnCompressor unCompressor;
+ private final Decoder timeDecoder =
+ Decoder.getDecoderByType(
+ TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()),
+ TSDataType.INT64);
+
+ protected Filter filter;
+
+ private final List<IPageReader> pageReaderList = new LinkedList<>();
+
+ /** A list of deleted intervals. */
+ private final List<List<TimeRange>> valueDeleteIntervalList;
+
+ /**
+ * constructor of ChunkReader.
+ *
+ * @param filter filter
+ */
+ public VectorChunkReader(Chunk timeChunk, List<Chunk> valueChunkList, Filter filter)
+ throws IOException {
+ this.filter = filter;
+ this.timeChunkDataBuffer = timeChunk.getData();
+ this.valueDeleteIntervalList = new ArrayList<>();
+ this.timeChunkHeader = timeChunk.getHeader();
+ this.unCompressor = IUnCompressor.getUnCompressor(timeChunkHeader.getCompressionType());
+ List<Statistics> valueChunkStatisticsList = new ArrayList<>();
+ valueChunkList.forEach(
+ chunk -> {
+ this.valueChunkHeaderList.add(chunk.getHeader());
+ this.valueChunkDataBufferList.add(chunk.getData());
+ valueChunkStatisticsList.add(chunk.getChunkStatistic());
+ valueDeleteIntervalList.add(chunk.getDeleteIntervalList());
+ });
+ initAllPageReaders(timeChunk.getChunkStatistic(), valueChunkStatisticsList);
+ }
+
+ private void initAllPageReaders(
+ Statistics timeChunkStatistics, List<Statistics> valueChunkStatisticsList)
+ throws IOException {
+ // construct next satisfied page header
+ while (timeChunkDataBuffer.remaining() > 0) {
+ // deserialize a PageHeader from chunkDataBuffer
+ PageHeader timePageHeader;
+ List<PageHeader> valuePageHeaderList = new ArrayList<>();
+ if ((timeChunkHeader.getChunkType() & 0x3F) == MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER) {
+ timePageHeader = PageHeader.deserializeFrom(timeChunkDataBuffer, timeChunkStatistics);
+ for (int i = 0; i < valueChunkDataBufferList.size(); i++) {
+ valuePageHeaderList.add(
+ PageHeader.deserializeFrom(
+ valueChunkDataBufferList.get(i), valueChunkStatisticsList.get(i)));
+ }
+ } else {
+ timePageHeader =
+ PageHeader.deserializeFrom(timeChunkDataBuffer, timeChunkHeader.getDataType());
+ for (int i = 0; i < valueChunkDataBufferList.size(); i++) {
+ valuePageHeaderList.add(
+ PageHeader.deserializeFrom(
+ valueChunkDataBufferList.get(i), valueChunkHeaderList.get(i).getDataType()));
+ }
+ }
+ // if the current page satisfies
+ if (pageSatisfied(timePageHeader)) {
+ pageReaderList.add(constructPageReaderForNextPage(timePageHeader, valuePageHeaderList));
+ } else {
+ skipBytesInStreamByLength(timePageHeader, valuePageHeaderList);
+ }
+ }
+ }
+
+ private boolean pageSatisfied(PageHeader pageHeader) {
+ return filter == null || filter.satisfy(pageHeader.getStatistics());
+ }
+
+ /** used for value page filter */
+ private boolean pageSatisfied(PageHeader pageHeader, List<TimeRange> valueDeleteInterval) {
+ if (valueDeleteInterval != null) {
+ for (TimeRange range : valueDeleteInterval) {
+ if (range.contains(pageHeader.getStartTime(), pageHeader.getEndTime())) {
+ return false;
+ }
+ if (range.overlaps(new TimeRange(pageHeader.getStartTime(), pageHeader.getEndTime()))) {
+ pageHeader.setModified(true);
+ }
+ }
+ }
+ return filter == null || filter.satisfy(pageHeader.getStatistics());
+ }
+
+ private VectorPageReader constructPageReaderForNextPage(
+ PageHeader timePageHeader, List<PageHeader> valuePageHeader) throws IOException {
+ PageInfo timePageInfo = new PageInfo();
+ getPageInfo(timePageHeader, timeChunkDataBuffer, timeChunkHeader, timePageInfo);
+ PageInfo valuePageInfo = new PageInfo();
+ List<PageHeader> valuePageHeaderList = new ArrayList<>();
+ List<ByteBuffer> valuePageDataList = new ArrayList<>();
+ List<TSDataType> valueDataTypeList = new ArrayList<>();
+ List<Decoder> valueDecoderList = new ArrayList<>();
+ for (int i = 0; i < valuePageHeader.size(); i++) {
+ if (pageSatisfied(valuePageHeader.get(i), valueDeleteIntervalList.get(i))) {
+ getPageInfo(
+ valuePageHeader.get(i),
+ valueChunkDataBufferList.get(i),
+ valueChunkHeaderList.get(i),
+ valuePageInfo);
+ valuePageHeaderList.add(valuePageInfo.pageHeader);
+ valuePageDataList.add(valuePageInfo.pageData);
+ valueDataTypeList.add(valuePageInfo.dataType);
+ valueDecoderList.add(valuePageInfo.decoder);
+ } else {
+ valueChunkDataBufferList
+ .get(i)
+ .position(
+ valueChunkDataBufferList.get(i).position()
+ + valuePageHeader.get(i).getCompressedSize());
+ valuePageHeaderList.add(valuePageHeader.get(i));
+ valuePageDataList.add(null);
+ valueDataTypeList.add(null);
+ valueDecoderList.add(null);
+ }
+ }
+ VectorPageReader vectorPageReader =
+ new VectorPageReader(
+ timePageHeader,
+ timePageInfo.pageData,
+ timeDecoder,
+ valuePageHeaderList,
+ valuePageDataList,
+ valueDataTypeList,
+ valueDecoderList,
+ filter);
+ vectorPageReader.setDeleteIntervalList(valueDeleteIntervalList);
+ return vectorPageReader;
+ }
+
+ private void getPageInfo(
+ PageHeader pageHeader, ByteBuffer chunkBuffer, ChunkHeader chunkHeader, PageInfo pageInfo)
+ throws IOException {
+ pageInfo.pageHeader = pageHeader;
+ pageInfo.dataType = chunkHeader.getDataType();
+ int compressedPageBodyLength = pageHeader.getCompressedSize();
+ byte[] compressedPageBody = new byte[compressedPageBodyLength];
+
+ // doesn't has a complete page body
+ if (compressedPageBodyLength > chunkBuffer.remaining()) {
+ throw new IOException(
+ "do not has a complete page body. Expected:"
+ + compressedPageBodyLength
+ + ". Actual:"
+ + chunkBuffer.remaining());
+ }
+
+ chunkBuffer.get(compressedPageBody);
+ pageInfo.decoder =
+ Decoder.getDecoderByType(chunkHeader.getEncodingType(), chunkHeader.getDataType());
+ byte[] uncompressedPageData = new byte[pageHeader.getUncompressedSize()];
+ try {
+ unCompressor.uncompress(
+ compressedPageBody, 0, compressedPageBodyLength, uncompressedPageData, 0);
+ } catch (Exception e) {
+ throw new IOException(
+ "Uncompress error! uncompress size: "
+ + pageHeader.getUncompressedSize()
+ + "compressed size: "
+ + pageHeader.getCompressedSize()
+ + "page header: "
+ + pageHeader
+ + e.getMessage());
+ }
+ pageInfo.pageData = ByteBuffer.wrap(uncompressedPageData);
+ }
+
+ private static class PageInfo {
+
+ PageHeader pageHeader;
+ ByteBuffer pageData;
+ TSDataType dataType;
+ Decoder decoder;
+ }
+
+ private void skipBytesInStreamByLength(
+ PageHeader timePageHeader, List<PageHeader> valuePageHeader) {
+ timeChunkDataBuffer.position(
+ timeChunkDataBuffer.position() + timePageHeader.getCompressedSize());
+ for (int i = 0; i < valuePageHeader.size(); i++) {
+ valueChunkDataBufferList
+ .get(i)
+ .position(
+ valueChunkDataBufferList.get(i).position()
+ + valuePageHeader.get(i).getCompressedSize());
+ }
+ }
+
+ @Override
+ public boolean hasNextSatisfiedPage() {
+ return !pageReaderList.isEmpty();
+ }
+
+ @Override
+ public BatchData nextPageData() throws IOException {
+ if (pageReaderList.isEmpty()) {
+ throw new IOException("No more page");
+ }
+ return pageReaderList.remove(0).getAllSatisfiedPageData();
+ }
+
+ @Override
+ public void close() throws IOException {}
+
+ @Override
+ public List<IPageReader> loadPageReaderList() {
+ return pageReaderList;
+ }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/TimePageReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/TimePageReader.java
new file mode 100644
index 0000000..aa39832
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/TimePageReader.java
@@ -0,0 +1,92 @@
+/*
+ * 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.tsfile.read.reader.page;
+
+import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.statistics.TimeStatistics;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+public class TimePageReader {
+
+ private final PageHeader pageHeader;
+
+ /** decoder for time column */
+ protected Decoder timeDecoder;
+
+ /** time column in memory */
+ protected ByteBuffer timeBuffer;
+
+ /** A list of deleted intervals. */
+ private List<TimeRange> deleteIntervalList;
+
+ private int deleteCursor = 0;
+
+ public TimePageReader(ByteBuffer pageData, Decoder timeDecoder) {
+ this(null, pageData, timeDecoder);
+ }
+
+ public TimePageReader(PageHeader pageHeader, ByteBuffer pageData, Decoder timeDecoder) {
+ this.timeDecoder = timeDecoder;
+ this.pageHeader = pageHeader;
+ this.timeBuffer = pageData;
+ }
+
+ public long[] nexTimeBatch() throws IOException {
+ long[] timeBatch = new long[(int) pageHeader.getStatistics().getCount()];
+ int index = 0;
+ while (timeDecoder.hasNext(timeBuffer)) {
+ timeBatch[index++] = timeDecoder.readLong(timeBuffer);
+ }
+ return timeBatch;
+ }
+
+ public TimeStatistics getStatistics() {
+ return (TimeStatistics) pageHeader.getStatistics();
+ }
+
+ public void setDeleteIntervalList(List<TimeRange> list) {
+ this.deleteIntervalList = list;
+ }
+
+ public List<TimeRange> getDeleteIntervalList() {
+ return deleteIntervalList;
+ }
+
+ public boolean isModified() {
+ return pageHeader.isModified();
+ }
+
+ protected boolean isDeleted(long timestamp) {
+ while (deleteIntervalList != null && deleteCursor < deleteIntervalList.size()) {
+ if (deleteIntervalList.get(deleteCursor).contains(timestamp)) {
+ return true;
+ } else if (deleteIntervalList.get(deleteCursor).getMax() < timestamp) {
+ deleteCursor++;
+ } else {
+ return false;
+ }
+ }
+ return false;
+ }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/ValuePageReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/ValuePageReader.java
new file mode 100644
index 0000000..dd5eea3
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/ValuePageReader.java
@@ -0,0 +1,218 @@
+/*
+ * 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.tsfile.read.reader.page;
+
+import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.BatchDataFactory;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+public class ValuePageReader {
+
+ private static final int MASK = 0x80;
+
+ private final PageHeader pageHeader;
+
+ private final TSDataType dataType;
+
+ /** decoder for value column */
+ private final Decoder valueDecoder;
+
+ private byte[] bitmap;
+
+ private int size;
+
+ /** value column in memory */
+ protected ByteBuffer valueBuffer;
+
+ /** A list of deleted intervals. */
+ private List<TimeRange> deleteIntervalList;
+
+ private int deleteCursor = 0;
+
+ public ValuePageReader(ByteBuffer pageData, TSDataType dataType, Decoder valueDecoder) {
+ this(null, pageData, dataType, valueDecoder);
+ }
+
+ public ValuePageReader(
+ PageHeader pageHeader, ByteBuffer pageData, TSDataType dataType, Decoder valueDecoder) {
+ this.dataType = dataType;
+ this.valueDecoder = valueDecoder;
+ this.pageHeader = pageHeader;
+ if (pageData != null) {
+ splitDataToBitmapAndValue(pageData);
+ }
+ this.valueBuffer = pageData;
+ }
+
+ private void splitDataToBitmapAndValue(ByteBuffer pageData) {
+ this.size = ReadWriteIOUtils.readInt(pageData);
+ this.bitmap = new byte[(size + 7) / 8];
+ pageData.get(bitmap);
+ this.valueBuffer = pageData.slice();
+ }
+
+ public BatchData nextBatch(long[] timeBatch, boolean ascending, Filter filter) {
+ BatchData pageData = BatchDataFactory.createBatchData(dataType, ascending, false);
+ for (int i = 0; i < timeBatch.length; i++) {
+ if (((bitmap[i / 8] & 0xFF) & (MASK >>> (i % 8))) == 0) {
+ continue;
+ }
+ long timestamp = timeBatch[i];
+ switch (dataType) {
+ case BOOLEAN:
+ boolean aBoolean = valueDecoder.readBoolean(valueBuffer);
+ if (!isDeleted(timestamp) && (filter == null || filter.satisfy(timestamp, aBoolean))) {
+ pageData.putBoolean(timestamp, aBoolean);
+ }
+ break;
+ case INT32:
+ int anInt = valueDecoder.readInt(valueBuffer);
+ if (!isDeleted(timestamp) && (filter == null || filter.satisfy(timestamp, anInt))) {
+ pageData.putInt(timestamp, anInt);
+ }
+ break;
+ case INT64:
+ long aLong = valueDecoder.readLong(valueBuffer);
+ if (!isDeleted(timestamp) && (filter == null || filter.satisfy(timestamp, aLong))) {
+ pageData.putLong(timestamp, aLong);
+ }
+ break;
+ case FLOAT:
+ float aFloat = valueDecoder.readFloat(valueBuffer);
+ if (!isDeleted(timestamp) && (filter == null || filter.satisfy(timestamp, aFloat))) {
+ pageData.putFloat(timestamp, aFloat);
+ }
+ break;
+ case DOUBLE:
+ double aDouble = valueDecoder.readDouble(valueBuffer);
+ if (!isDeleted(timestamp) && (filter == null || filter.satisfy(timestamp, aDouble))) {
+ pageData.putDouble(timestamp, aDouble);
+ }
+ break;
+ case TEXT:
+ Binary aBinary = valueDecoder.readBinary(valueBuffer);
+ if (!isDeleted(timestamp) && (filter == null || filter.satisfy(timestamp, aBinary))) {
+ pageData.putBinary(timestamp, aBinary);
+ }
+ break;
+ default:
+ throw new UnSupportedDataTypeException(String.valueOf(dataType));
+ }
+ }
+ return pageData.flip();
+ }
+
+ public TsPrimitiveType[] nextValueBatch(long[] timeBatch) {
+ TsPrimitiveType[] valueBatch = new TsPrimitiveType[size];
+ if (valueBuffer == null) {
+ return valueBatch;
+ }
+ for (int i = 0; i < size; i++) {
+ if (((bitmap[i / 8] & 0xFF) & (MASK >>> (i % 8))) == 0) {
+ continue;
+ }
+ switch (dataType) {
+ case BOOLEAN:
+ boolean aBoolean = valueDecoder.readBoolean(valueBuffer);
+ if (!isDeleted(timeBatch[i])) {
+ valueBatch[i] = new TsPrimitiveType.TsBoolean(aBoolean);
+ }
+ break;
+ case INT32:
+ int anInt = valueDecoder.readInt(valueBuffer);
+ if (!isDeleted(timeBatch[i])) {
+ valueBatch[i] = new TsPrimitiveType.TsInt(anInt);
+ }
+ break;
+ case INT64:
+ long aLong = valueDecoder.readLong(valueBuffer);
+ if (!isDeleted(timeBatch[i])) {
+ valueBatch[i] = new TsPrimitiveType.TsLong(aLong);
+ }
+ break;
+ case FLOAT:
+ float aFloat = valueDecoder.readFloat(valueBuffer);
+ if (!isDeleted(timeBatch[i])) {
+ valueBatch[i] = new TsPrimitiveType.TsFloat(aFloat);
+ }
+ break;
+ case DOUBLE:
+ double aDouble = valueDecoder.readDouble(valueBuffer);
+ if (!isDeleted(timeBatch[i])) {
+ valueBatch[i] = new TsPrimitiveType.TsDouble(aDouble);
+ }
+ break;
+ case TEXT:
+ Binary aBinary = valueDecoder.readBinary(valueBuffer);
+ if (!isDeleted(timeBatch[i])) {
+ valueBatch[i] = new TsPrimitiveType.TsBinary(aBinary);
+ }
+ break;
+ default:
+ throw new UnSupportedDataTypeException(String.valueOf(dataType));
+ }
+ }
+ return valueBatch;
+ }
+
+ public Statistics getStatistics() {
+ return pageHeader.getStatistics();
+ }
+
+ public void setDeleteIntervalList(List<TimeRange> list) {
+ this.deleteIntervalList = list;
+ }
+
+ public List<TimeRange> getDeleteIntervalList() {
+ return deleteIntervalList;
+ }
+
+ public boolean isModified() {
+ return pageHeader.isModified();
+ }
+
+ protected boolean isDeleted(long timestamp) {
+ while (deleteIntervalList != null && deleteCursor < deleteIntervalList.size()) {
+ if (deleteIntervalList.get(deleteCursor).contains(timestamp)) {
+ return true;
+ } else if (deleteIntervalList.get(deleteCursor).getMax() < timestamp) {
+ deleteCursor++;
+ } else {
+ return false;
+ }
+ }
+ return false;
+ }
+
+ public TSDataType getDataType() {
+ return dataType;
+ }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/VectorChunkReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/VectorChunkReader.java
new file mode 100644
index 0000000..b4665fc
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/VectorChunkReader.java
@@ -0,0 +1,21 @@
+/*
+ * 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.tsfile.read.reader.page;
+
+public class VectorChunkReader {}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/VectorPageReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/VectorPageReader.java
new file mode 100644
index 0000000..eeb095d
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/VectorPageReader.java
@@ -0,0 +1,122 @@
+/*
+ * 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.tsfile.read.reader.page;
+
+import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.BatchDataFactory;
+import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.read.reader.IPageReader;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+public class VectorPageReader implements IPageReader {
+
+ private final TimePageReader timePageReader;
+ private final List<ValuePageReader> valuePageReaderList;
+ private final int valueCount;
+ private Filter filter;
+ private boolean isModified;
+
+ public VectorPageReader(
+ PageHeader timePageHeader,
+ ByteBuffer timePageData,
+ Decoder timeDecoder,
+ List<PageHeader> valuePageHeaderList,
+ List<ByteBuffer> valuePageDataList,
+ List<TSDataType> valueDataTypeList,
+ List<Decoder> valueDecoderList,
+ Filter filter) {
+ timePageReader = new TimePageReader(timePageHeader, timePageData, timeDecoder);
+ isModified = timePageReader.isModified();
+ valuePageReaderList = new ArrayList<>(valuePageHeaderList.size());
+ for (int i = 0; i < valuePageHeaderList.size(); i++) {
+ ValuePageReader valuePageReader =
+ new ValuePageReader(
+ valuePageHeaderList.get(i),
+ valuePageDataList.get(i),
+ valueDataTypeList.get(i),
+ valueDecoderList.get(i));
+ valuePageReaderList.add(valuePageReader);
+ isModified = isModified && valuePageReader.isModified();
+ }
+ this.filter = filter;
+ this.valueCount = valuePageReaderList.size();
+ }
+
+ @Override
+ public BatchData getAllSatisfiedPageData(boolean ascending) throws IOException {
+ long[] timeBatch = timePageReader.nexTimeBatch();
+ if (valuePageReaderList.size() == 1) {
+ return valuePageReaderList.get(0).nextBatch(timeBatch, ascending, filter);
+ }
+ List<TsPrimitiveType[]> valueBatchList = new ArrayList<>(valueCount);
+ for (ValuePageReader valuePageReader : valuePageReaderList) {
+ valueBatchList.add(valuePageReader.nextValueBatch(timeBatch));
+ }
+ BatchData pageData = BatchDataFactory.createBatchData(TSDataType.VECTOR, ascending, false);
+ boolean isNull;
+ for (int i = 0; i < timeBatch.length; i++) {
+ isNull = true;
+ TsPrimitiveType[] v = new TsPrimitiveType[valueCount];
+ for (int j = 0; j < v.length; j++) {
+ v[j] = valueBatchList.get(j)[i];
+ if (v[j] != null) {
+ isNull = false;
+ }
+ }
+ // TODO fix value filter v[0].getValue()
+ if (!isNull && (filter == null || filter.satisfy(timeBatch[i], v[0].getValue()))) {
+ pageData.putVector(timeBatch[i], v);
+ }
+ }
+ return pageData;
+ }
+
+ public void setDeleteIntervalList(List<List<TimeRange>> list) {
+ for (int i = 0; i < valueCount; i++) {
+ valuePageReaderList.get(i).setDeleteIntervalList(list.get(i));
+ }
+ }
+
+ @Override
+ public Statistics getStatistics() {
+ return valuePageReaderList.size() == 1
+ ? valuePageReaderList.get(0).getStatistics()
+ : timePageReader.getStatistics();
+ }
+
+ @Override
+ public void setFilter(Filter filter) {
+ this.filter = filter;
+ }
+
+ @Override
+ public boolean isModified() {
+ return isModified;
+ }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/AbstractFileSeriesReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/AbstractFileSeriesReader.java
index 52489a4..e0e4444 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/AbstractFileSeriesReader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/AbstractFileSeriesReader.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.tsfile.read.reader.series;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.read.common.BatchData;
import org.apache.iotdb.tsfile.read.controller.IChunkLoader;
import org.apache.iotdb.tsfile.read.filter.basic.Filter;
@@ -33,7 +33,7 @@
public abstract class AbstractFileSeriesReader implements IBatchReader {
protected IChunkLoader chunkLoader;
- protected List<ChunkMetadata> chunkMetadataList;
+ protected List<IChunkMetadata> chunkMetadataList;
protected ChunkReader chunkReader;
private int chunkToRead;
@@ -41,7 +41,7 @@
/** constructor of FileSeriesReader. */
public AbstractFileSeriesReader(
- IChunkLoader chunkLoader, List<ChunkMetadata> chunkMetadataList, Filter filter) {
+ IChunkLoader chunkLoader, List<IChunkMetadata> chunkMetadataList, Filter filter) {
this.chunkLoader = chunkLoader;
this.chunkMetadataList = chunkMetadataList;
this.filter = filter;
@@ -59,7 +59,7 @@
// current chunk does not have additional batch, init new chunk reader
while (chunkToRead < chunkMetadataList.size()) {
- ChunkMetadata chunkMetaData = nextChunkMeta();
+ IChunkMetadata chunkMetaData = nextChunkMeta();
if (chunkSatisfied(chunkMetaData)) {
// chunk metadata satisfy the condition
initChunkReader(chunkMetaData);
@@ -77,16 +77,16 @@
return chunkReader.nextPageData();
}
- protected abstract void initChunkReader(ChunkMetadata chunkMetaData) throws IOException;
+ protected abstract void initChunkReader(IChunkMetadata chunkMetaData) throws IOException;
- protected abstract boolean chunkSatisfied(ChunkMetadata chunkMetaData);
+ protected abstract boolean chunkSatisfied(IChunkMetadata chunkMetaData);
@Override
public void close() throws IOException {
chunkLoader.close();
}
- private ChunkMetadata nextChunkMeta() {
+ private IChunkMetadata nextChunkMeta() {
return chunkMetadataList.get(chunkToRead++);
}
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/EmptyFileSeriesReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/EmptyFileSeriesReader.java
index 527d6c3..0316987 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/EmptyFileSeriesReader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/EmptyFileSeriesReader.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.tsfile.read.reader.series;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.read.common.BatchData;
/** this is for those series which has no data points */
@@ -31,12 +31,12 @@
}
@Override
- protected void initChunkReader(ChunkMetadata chunkMetaData) {
+ protected void initChunkReader(IChunkMetadata chunkMetaData) {
// do nothing
}
@Override
- protected boolean chunkSatisfied(ChunkMetadata chunkMetaData) {
+ protected boolean chunkSatisfied(IChunkMetadata chunkMetaData) {
return false;
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReader.java
index e48dca7..10dde36 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReader.java
@@ -19,6 +19,7 @@
package org.apache.iotdb.tsfile.read.reader.series;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.read.common.Chunk;
import org.apache.iotdb.tsfile.read.controller.IChunkLoader;
import org.apache.iotdb.tsfile.read.filter.basic.Filter;
@@ -34,18 +35,18 @@
public class FileSeriesReader extends AbstractFileSeriesReader {
public FileSeriesReader(
- IChunkLoader chunkLoader, List<ChunkMetadata> chunkMetadataList, Filter filter) {
+ IChunkLoader chunkLoader, List<IChunkMetadata> chunkMetadataList, Filter filter) {
super(chunkLoader, chunkMetadataList, filter);
}
@Override
- protected void initChunkReader(ChunkMetadata chunkMetaData) throws IOException {
- Chunk chunk = chunkLoader.loadChunk(chunkMetaData);
+ protected void initChunkReader(IChunkMetadata chunkMetaData) throws IOException {
+ Chunk chunk = chunkLoader.loadChunk((ChunkMetadata) chunkMetaData);
this.chunkReader = new ChunkReader(chunk, filter);
}
@Override
- protected boolean chunkSatisfied(ChunkMetadata chunkMetaData) {
+ protected boolean chunkSatisfied(IChunkMetadata chunkMetaData) {
return filter == null || filter.satisfy(chunkMetaData.getStatistics());
}
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReaderByTimestamp.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReaderByTimestamp.java
index 504aad9..482a192 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReaderByTimestamp.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReaderByTimestamp.java
@@ -19,6 +19,7 @@
package org.apache.iotdb.tsfile.read.reader.series;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.common.BatchData;
import org.apache.iotdb.tsfile.read.common.Chunk;
@@ -36,7 +37,7 @@
public class FileSeriesReaderByTimestamp {
protected IChunkLoader chunkLoader;
- protected List<ChunkMetadata> chunkMetadataList;
+ protected List<IChunkMetadata> chunkMetadataList;
private int currentChunkIndex = 0;
private ChunkReader chunkReader;
@@ -45,7 +46,7 @@
/** init with chunkLoader and chunkMetaDataList. */
public FileSeriesReaderByTimestamp(
- IChunkLoader chunkLoader, List<ChunkMetadata> chunkMetadataList) {
+ IChunkLoader chunkLoader, List<IChunkMetadata> chunkMetadataList) {
this.chunkLoader = chunkLoader;
this.chunkMetadataList = chunkMetadataList;
currentTimestamp = Long.MIN_VALUE;
@@ -133,7 +134,7 @@
private boolean constructNextSatisfiedChunkReader() throws IOException {
while (currentChunkIndex < chunkMetadataList.size()) {
- ChunkMetadata chunkMetaData = chunkMetadataList.get(currentChunkIndex++);
+ IChunkMetadata chunkMetaData = chunkMetadataList.get(currentChunkIndex++);
if (chunkSatisfied(chunkMetaData)) {
initChunkReader(chunkMetaData);
((ChunkReaderByTimestamp) chunkReader).setCurrentTimestamp(currentTimestamp);
@@ -143,12 +144,12 @@
return false;
}
- private void initChunkReader(ChunkMetadata chunkMetaData) throws IOException {
- Chunk chunk = chunkLoader.loadChunk(chunkMetaData);
+ private void initChunkReader(IChunkMetadata chunkMetaData) throws IOException {
+ Chunk chunk = chunkLoader.loadChunk((ChunkMetadata) chunkMetaData);
this.chunkReader = new ChunkReaderByTimestamp(chunk);
}
- private boolean chunkSatisfied(ChunkMetadata chunkMetaData) {
+ private boolean chunkSatisfied(IChunkMetadata chunkMetaData) {
return chunkMetaData.getEndTime() >= currentTimestamp;
}
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/Binary.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/Binary.java
index 9d73967..0411e40 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/Binary.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/Binary.java
@@ -30,6 +30,8 @@
public class Binary implements Comparable<Binary>, Serializable {
private static final long serialVersionUID = 6394197743397020735L;
+ public static final Binary EMPTY_VALUE = new Binary("");
+
private byte[] values;
/** if the bytes v is modified, the modification is visible to this binary. */
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/BitMap.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/BitMap.java
new file mode 100644
index 0000000..91affdb
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/BitMap.java
@@ -0,0 +1,121 @@
+/*
+ * 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.tsfile.utils;
+
+import java.util.Arrays;
+
+public class BitMap {
+ private static final byte[] BIT_UTIL = new byte[] {1, 2, 4, 8, 16, 32, 64, -128};
+ private static final byte[] UNMARK_BIT_UTIL =
+ new byte[] {
+ (byte) 0XFE, // 11111110
+ (byte) 0XFD, // 11111101
+ (byte) 0XFB, // 11111011
+ (byte) 0XF7, // 11110111
+ (byte) 0XEF, // 11101111
+ (byte) 0XDF, // 11011111
+ (byte) 0XBF, // 10111111
+ (byte) 0X7F // 01111111
+ };
+
+ private byte[] bits;
+ private int size;
+
+ /** Initialize a BitMap with given size. */
+ public BitMap(int size) {
+ this.size = size;
+ bits = new byte[size / Byte.SIZE + 1];
+ Arrays.fill(bits, (byte) 0);
+ }
+
+ /** Initialize a BitMap with given size and bytes. */
+ public BitMap(int size, byte[] bits) {
+ this.size = size;
+ this.bits = bits;
+ }
+
+ public byte[] getByteArray() {
+ return this.bits;
+ }
+
+ public int getSize() {
+ return this.size;
+ }
+
+ /** returns the value of the bit with the specified index. */
+ public boolean isMarked(int position) {
+ return (bits[position / Byte.SIZE] & BIT_UTIL[position % Byte.SIZE]) != 0;
+ }
+
+ /** mark as 1 at the given bit position. */
+ public void mark(int position) {
+ bits[position / Byte.SIZE] |= BIT_UTIL[position % Byte.SIZE];
+ }
+
+ /** mark as 0 at all positions. */
+ public void reset() {
+ Arrays.fill(bits, (byte) 0);
+ }
+
+ public void unmark(int position) {
+ bits[position / Byte.SIZE] &= UNMARK_BIT_UTIL[position % Byte.SIZE];
+ }
+
+ /** whether all bits are zero, i.e., no Null value */
+ public boolean isAllUnmarked() {
+ int j;
+ for (j = 0; j < size / Byte.SIZE; j++) {
+ if (bits[j] != (byte) 0) {
+ return false;
+ }
+ }
+ for (j = 0; j < size % Byte.SIZE; j++) {
+ if ((bits[size / Byte.SIZE] & BIT_UTIL[j]) != 0) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ /** whether all bits are one, i.e., all are Null */
+ public boolean isAllMarked() {
+ int j;
+ for (j = 0; j < size / Byte.SIZE; j++) {
+ if (bits[j] != (byte) 0XFF) {
+ return false;
+ }
+ }
+ for (j = 0; j < size % Byte.SIZE; j++) {
+ if ((bits[size / Byte.SIZE] & BIT_UTIL[j]) == 0) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder res = new StringBuilder();
+ for (int i = 0; i < size; i++) {
+ res.append(isMarked(i) ? 1 : 0);
+ }
+ return res.toString();
+ }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/PublicBAOS.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/PublicBAOS.java
index 881db03..61efe8e 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/PublicBAOS.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/PublicBAOS.java
@@ -86,4 +86,8 @@
public int size() {
return count;
}
+
+ public void truncate(int size) {
+ count = size;
+ }
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/TsPrimitiveType.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/TsPrimitiveType.java
index 73b01d2..7bfd218 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/TsPrimitiveType.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/TsPrimitiveType.java
@@ -22,6 +22,7 @@
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import java.io.Serializable;
+import java.util.Arrays;
public abstract class TsPrimitiveType implements Serializable {
@@ -45,6 +46,8 @@
return new TsPrimitiveType.TsDouble((double) v);
case TEXT:
return new TsPrimitiveType.TsBinary((Binary) v);
+ case VECTOR:
+ return new TsPrimitiveType.TsVector((TsPrimitiveType[]) v);
default:
throw new UnSupportedDataTypeException("Unsupported data type:" + dataType);
}
@@ -74,6 +77,10 @@
throw new UnsupportedOperationException("getBinary() is not supported for current sub-class");
}
+ public TsPrimitiveType[] getVector() {
+ throw new UnsupportedOperationException("getVector() is not supported for current sub-class");
+ }
+
public void setBoolean(boolean val) {
throw new UnsupportedOperationException("setBoolean() is not supported for current sub-class");
}
@@ -98,6 +105,10 @@
throw new UnsupportedOperationException("setBinary() is not supported for current sub-class");
}
+ public void setVector(TsPrimitiveType[] val) {
+ throw new UnsupportedOperationException("setVector() is not supported for current sub-class");
+ }
+
/**
* get the size of one instance of current class.
*
@@ -462,4 +473,81 @@
return false;
}
}
+
+ public static class TsVector extends TsPrimitiveType {
+
+ private TsPrimitiveType[] values;
+
+ public TsVector(TsPrimitiveType[] values) {
+ this.values = values;
+ }
+
+ @Override
+ public TsPrimitiveType[] getVector() {
+ return values;
+ }
+
+ @Override
+ public void setVector(TsPrimitiveType[] vals) {
+ this.values = vals;
+ }
+
+ @Override
+ public int getSize() {
+ int size = 0;
+ for (TsPrimitiveType type : values) {
+ if (type != null) {
+ size += type.getSize();
+ }
+ }
+ // object header + array object header
+ return 4 + 4 + size;
+ }
+
+ @Override
+ public Object getValue() {
+ return getVector();
+ }
+
+ @Override
+ public String getStringValue() {
+ StringBuilder builder = new StringBuilder("[");
+ builder.append(values[0] == null ? "null" : values[0].getStringValue());
+ for (int i = 1; i < values.length; i++) {
+ builder.append(", ").append(values[i] == null ? "null" : values[i].getStringValue());
+ }
+ builder.append("]");
+ return builder.toString();
+ }
+
+ @Override
+ public TSDataType getDataType() {
+ return TSDataType.VECTOR;
+ }
+
+ @Override
+ public int hashCode() {
+ return Arrays.hashCode(values);
+ }
+
+ @Override
+ public boolean equals(Object anObject) {
+ if (this == anObject) {
+ return true;
+ }
+ if (anObject instanceof TsVector) {
+ TsVector anotherTs = (TsVector) anObject;
+ if (anotherTs.values.length != this.values.length) {
+ return false;
+ }
+ for (int i = 0; i < this.values.length; i++) {
+ if (!values[i].equals(anotherTs.values[i])) {
+ return false;
+ }
+ }
+ return true;
+ }
+ return false;
+ }
+ }
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/TsFileWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/TsFileWriter.java
index 8ce3f01..69bbab8 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/TsFileWriter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/TsFileWriter.java
@@ -28,7 +28,7 @@
import org.apache.iotdb.tsfile.write.record.TSRecord;
import org.apache.iotdb.tsfile.write.record.Tablet;
import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.Schema;
import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
@@ -150,7 +150,8 @@
}
}
- public void registerDeviceTemplate(String templateName, Map<String, MeasurementSchema> template) {
+ public void registerDeviceTemplate(
+ String templateName, Map<String, IMeasurementSchema> template) {
schema.registerDeviceTemplate(templateName, template);
}
@@ -158,7 +159,7 @@
schema.registerDevice(deviceId, templateName);
}
- public void registerTimeseries(Path path, MeasurementSchema measurementSchema)
+ public void registerTimeseries(Path path, IMeasurementSchema measurementSchema)
throws WriteProcessException {
if (schema.containsTimeseries(path)) {
throw new WriteProcessException("given timeseries has exists! " + path);
@@ -190,7 +191,7 @@
groupWriter.tryToAddSeriesWriter(schema.getSeriesSchema(path), pageSize);
} else if (schema.getDeviceTemplates() != null && schema.getDeviceTemplates().size() == 1) {
// use the default template without needing to register device
- Map<String, MeasurementSchema> template =
+ Map<String, IMeasurementSchema> template =
schema.getDeviceTemplates().entrySet().iterator().next().getValue();
if (template.containsKey(path.getMeasurement())) {
groupWriter.tryToAddSeriesWriter(template.get(path.getMeasurement()), pageSize);
@@ -220,14 +221,14 @@
String deviceId = tablet.deviceId;
// add all SeriesWriter of measurements in this Tablet to this ChunkGroupWriter
- for (MeasurementSchema timeseries : tablet.getSchemas()) {
+ for (IMeasurementSchema timeseries : tablet.getSchemas()) {
String measurementId = timeseries.getMeasurementId();
Path path = new Path(deviceId, measurementId);
if (schema.containsTimeseries(path)) {
groupWriter.tryToAddSeriesWriter(schema.getSeriesSchema(path), pageSize);
} else if (schema.getDeviceTemplates() != null && schema.getDeviceTemplates().size() == 1) {
// use the default template without needing to register device
- Map<String, MeasurementSchema> template =
+ Map<String, IMeasurementSchema> template =
schema.getDeviceTemplates().entrySet().iterator().next().getValue();
if (template.containsKey(path.getMeasurement())) {
groupWriter.tryToAddSeriesWriter(template.get(path.getMeasurement()), pageSize);
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ChunkGroupWriterImpl.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ChunkGroupWriterImpl.java
index 1dbc9f5..10e3044 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ChunkGroupWriterImpl.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ChunkGroupWriterImpl.java
@@ -25,7 +25,7 @@
import org.apache.iotdb.tsfile.utils.Binary;
import org.apache.iotdb.tsfile.write.record.Tablet;
import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
import org.slf4j.Logger;
@@ -51,7 +51,7 @@
}
@Override
- public void tryToAddSeriesWriter(MeasurementSchema schema, int pageSizeThreshold) {
+ public void tryToAddSeriesWriter(IMeasurementSchema schema, int pageSizeThreshold) {
if (!chunkWriters.containsKey(schema.getMeasurementId())) {
IChunkWriter seriesWriter = new ChunkWriterImpl(schema);
this.chunkWriters.put(schema.getMeasurementId(), seriesWriter);
@@ -72,7 +72,7 @@
@Override
public void write(Tablet tablet) throws WriteProcessException {
- List<MeasurementSchema> timeseries = tablet.getSchemas();
+ List<IMeasurementSchema> timeseries = tablet.getSchemas();
for (int i = 0; i < timeseries.size(); i++) {
String measurementId = timeseries.get(i).getMeasurementId();
TSDataType dataType = timeseries.get(i).getType();
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ChunkWriterImpl.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ChunkWriterImpl.java
index 2c00f4a..078df52 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ChunkWriterImpl.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ChunkWriterImpl.java
@@ -30,7 +30,7 @@
import org.apache.iotdb.tsfile.utils.PublicBAOS;
import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
import org.apache.iotdb.tsfile.write.page.PageWriter;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
import org.slf4j.Logger;
@@ -45,7 +45,7 @@
private static final Logger logger = LoggerFactory.getLogger(ChunkWriterImpl.class);
- private MeasurementSchema measurementSchema;
+ private IMeasurementSchema measurementSchema;
private ICompressor compressor;
@@ -92,7 +92,7 @@
private Statistics<?> firstPageStatistics;
/** @param schema schema of this measurement */
- public ChunkWriterImpl(MeasurementSchema schema) {
+ public ChunkWriterImpl(IMeasurementSchema schema) {
this.measurementSchema = schema;
this.compressor = ICompressor.getCompressor(schema.getCompressor());
this.pageBuffer = new PublicBAOS();
@@ -115,7 +115,7 @@
checkSdtEncoding();
}
- public ChunkWriterImpl(MeasurementSchema schema, boolean isMerging) {
+ public ChunkWriterImpl(IMeasurementSchema schema, boolean isMerging) {
this(schema);
this.isMerging = isMerging;
}
@@ -145,7 +145,7 @@
}
@Override
- public void write(long time, long value) {
+ public void write(long time, long value, boolean isNull) {
// store last point for sdtEncoding, it still needs to go through encoding process
// in case it exceeds compdev and needs to store second last point
if (!isSdtEncoding || sdtEncoder.encodeLong(time, value)) {
@@ -160,7 +160,7 @@
}
@Override
- public void write(long time, int value) {
+ public void write(long time, int value, boolean isNull) {
if (!isSdtEncoding || sdtEncoder.encodeInt(time, value)) {
pageWriter.write(
isSdtEncoding ? sdtEncoder.getTime() : time,
@@ -173,13 +173,13 @@
}
@Override
- public void write(long time, boolean value) {
+ public void write(long time, boolean value, boolean isNull) {
pageWriter.write(time, value);
checkPageSizeAndMayOpenANewPage();
}
@Override
- public void write(long time, float value) {
+ public void write(long time, float value, boolean isNull) {
if (!isSdtEncoding || sdtEncoder.encodeFloat(time, value)) {
pageWriter.write(
isSdtEncoding ? sdtEncoder.getTime() : time,
@@ -193,7 +193,7 @@
}
@Override
- public void write(long time, double value) {
+ public void write(long time, double value, boolean isNull) {
if (!isSdtEncoding || sdtEncoder.encodeDouble(time, value)) {
pageWriter.write(
isSdtEncoding ? sdtEncoder.getTime() : time,
@@ -206,12 +206,17 @@
}
@Override
- public void write(long time, Binary value) {
+ public void write(long time, Binary value, boolean isNull) {
pageWriter.write(time, value);
checkPageSizeAndMayOpenANewPage();
}
@Override
+ public void write(long time) {
+ throw new IllegalStateException("write time method is not implemented in common chunk writer");
+ }
+
+ @Override
public void write(long[] timestamps, int[] values, int batchSize) {
if (isSdtEncoding) {
batchSize = sdtEncoder.encode(timestamps, values, batchSize);
@@ -412,7 +417,7 @@
* @param statistics the chunk statistics
* @throws IOException exception in IO
*/
- public void writeAllPagesOfChunkToTsFile(TsFileIOWriter writer, Statistics<?> statistics)
+ private void writeAllPagesOfChunkToTsFile(TsFileIOWriter writer, Statistics<?> statistics)
throws IOException {
if (statistics.getCount() == 0) {
return;
@@ -420,13 +425,14 @@
// start to write this column chunk
writer.startFlushChunk(
- measurementSchema,
+ measurementSchema.getMeasurementId(),
compressor.getType(),
measurementSchema.getType(),
measurementSchema.getEncodingType(),
statistics,
pageBuffer.size(),
- numOfPages);
+ numOfPages,
+ 0);
long dataOffset = writer.getPos();
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/IChunkGroupWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/IChunkGroupWriter.java
index 0c73fbb..9dee2ad 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/IChunkGroupWriter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/IChunkGroupWriter.java
@@ -21,7 +21,7 @@
import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
import org.apache.iotdb.tsfile.write.record.Tablet;
import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
import java.io.IOException;
@@ -77,7 +77,7 @@
* @param measurementSchema a measurement descriptor containing the message of the series
* @param pageSize the specified page size
*/
- void tryToAddSeriesWriter(MeasurementSchema measurementSchema, int pageSize);
+ void tryToAddSeriesWriter(IMeasurementSchema measurementSchema, int pageSize);
/**
* get the serialized size of current chunkGroup header + all chunks. Notice, the value does not
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/IChunkWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/IChunkWriter.java
index cab9615..963ef64 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/IChunkWriter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/IChunkWriter.java
@@ -28,22 +28,25 @@
public interface IChunkWriter {
/** write a time value pair. */
- void write(long time, int value);
+ void write(long time, int value, boolean isNull);
/** write a time value pair. */
- void write(long time, long value);
+ void write(long time, long value, boolean isNull);
/** write a time value pair. */
- void write(long time, boolean value);
+ void write(long time, boolean value, boolean isNull);
/** write a time value pair. */
- void write(long time, float value);
+ void write(long time, float value, boolean isNull);
/** write a time value pair. */
- void write(long time, double value);
+ void write(long time, double value, boolean isNull);
/** write a time value pair. */
- void write(long time, Binary value);
+ void write(long time, Binary value, boolean isNull);
+
+ /** write a time. */
+ void write(long time);
/** write time series */
void write(long[] timestamps, int[] values, int batchSize);
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/TimeChunkWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/TimeChunkWriter.java
new file mode 100644
index 0000000..522eff5
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/TimeChunkWriter.java
@@ -0,0 +1,255 @@
+/*
+ * 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.tsfile.write.chunk;
+
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.compress.ICompressor;
+import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+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.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.file.metadata.statistics.TimeStatistics;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.write.page.TimePageWriter;
+import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+public class TimeChunkWriter {
+
+ private static final Logger logger = LoggerFactory.getLogger(TimeChunkWriter.class);
+
+ private final String measurementId;
+
+ private final TSEncoding encodingType;
+
+ private final CompressionType compressionType;
+
+ /** all pages of this chunk. */
+ private final PublicBAOS pageBuffer;
+
+ private int numOfPages;
+
+ /** write data into current page */
+ private TimePageWriter pageWriter;
+
+ /** page size threshold. */
+ private final long pageSizeThreshold;
+
+ private final int maxNumberOfPointsInPage;
+
+ /** value count in current page. */
+ private int valueCountInOnePageForNextCheck;
+
+ // initial value for valueCountInOnePageForNextCheck
+ private static final int MINIMUM_RECORD_COUNT_FOR_CHECK = 1500;
+
+ /** statistic of this chunk. */
+ private TimeStatistics statistics;
+
+ /** first page info */
+ private int sizeWithoutStatistic;
+
+ private Statistics<?> firstPageStatistics;
+
+ public TimeChunkWriter(
+ String measurementId,
+ CompressionType compressionType,
+ TSEncoding encodingType,
+ Encoder timeEncoder) {
+ this.measurementId = measurementId;
+ this.encodingType = encodingType;
+ this.compressionType = compressionType;
+ this.pageBuffer = new PublicBAOS();
+
+ this.pageSizeThreshold = TSFileDescriptor.getInstance().getConfig().getPageSizeInByte();
+ this.maxNumberOfPointsInPage =
+ TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage();
+ // initial check of memory usage. So that we have enough data to make an initial prediction
+ this.valueCountInOnePageForNextCheck = MINIMUM_RECORD_COUNT_FOR_CHECK;
+
+ // init statistics for this chunk and page
+ this.statistics = new TimeStatistics();
+
+ this.pageWriter = new TimePageWriter(timeEncoder, ICompressor.getCompressor(compressionType));
+ }
+
+ public void write(long time) {
+ pageWriter.write(time);
+ }
+
+ public void write(long[] timestamps, int batchSize) {
+ pageWriter.write(timestamps, batchSize);
+ }
+
+ /**
+ * check occupied memory size, if it exceeds the PageSize threshold, construct a page and put it
+ * to pageBuffer
+ */
+ public boolean checkPageSizeAndMayOpenANewPage() {
+ if (pageWriter.getPointNumber() == maxNumberOfPointsInPage) {
+ logger.debug("current line count reaches the upper bound, write page {}", measurementId);
+ return true;
+ } else if (pageWriter.getPointNumber()
+ >= valueCountInOnePageForNextCheck) { // need to check memory size
+ // not checking the memory used for every value
+ long currentPageSize = pageWriter.estimateMaxMemSize();
+ if (currentPageSize > pageSizeThreshold) { // memory size exceeds threshold
+ // we will write the current page
+ logger.debug(
+ "enough size, write page {}, pageSizeThreshold:{}, currentPateSize:{}, valueCountInOnePage:{}",
+ measurementId,
+ pageSizeThreshold,
+ currentPageSize,
+ pageWriter.getPointNumber());
+ valueCountInOnePageForNextCheck = MINIMUM_RECORD_COUNT_FOR_CHECK;
+ return true;
+ } else {
+ // reset the valueCountInOnePageForNextCheck for the next page
+ valueCountInOnePageForNextCheck =
+ (int) (((float) pageSizeThreshold / currentPageSize) * pageWriter.getPointNumber());
+ }
+ }
+ return false;
+ }
+
+ public void writePageToPageBuffer() {
+ try {
+ if (numOfPages == 0) { // record the firstPageStatistics
+ this.firstPageStatistics = pageWriter.getStatistics();
+ this.sizeWithoutStatistic = pageWriter.writePageHeaderAndDataIntoBuff(pageBuffer, true);
+ } else if (numOfPages == 1) { // put the firstPageStatistics into pageBuffer
+ byte[] b = pageBuffer.toByteArray();
+ pageBuffer.reset();
+ pageBuffer.write(b, 0, this.sizeWithoutStatistic);
+ firstPageStatistics.serialize(pageBuffer);
+ pageBuffer.write(b, this.sizeWithoutStatistic, b.length - this.sizeWithoutStatistic);
+ pageWriter.writePageHeaderAndDataIntoBuff(pageBuffer, false);
+ firstPageStatistics = null;
+ } else {
+ pageWriter.writePageHeaderAndDataIntoBuff(pageBuffer, false);
+ }
+
+ // update statistics of this chunk
+ numOfPages++;
+ this.statistics.mergeStatistics(pageWriter.getStatistics());
+ } catch (IOException e) {
+ logger.error("meet error in pageWriter.writePageHeaderAndDataIntoBuff,ignore this page:", e);
+ } finally {
+ // clear start time stamp for next initializing
+ pageWriter.reset();
+ }
+ }
+
+ public void writeToFileWriter(TsFileIOWriter tsfileWriter) throws IOException {
+ sealCurrentPage();
+ writeAllPagesOfChunkToTsFile(tsfileWriter);
+
+ // reinit this chunk writer
+ pageBuffer.reset();
+ numOfPages = 0;
+ firstPageStatistics = null;
+ this.statistics = new TimeStatistics();
+ }
+
+ public long estimateMaxSeriesMemSize() {
+ return pageBuffer.size()
+ + pageWriter.estimateMaxMemSize()
+ + PageHeader.estimateMaxPageHeaderSizeWithoutStatistics()
+ + pageWriter.getStatistics().getSerializedSize();
+ }
+
+ public long getCurrentChunkSize() {
+ if (pageBuffer.size() == 0) {
+ return 0;
+ }
+ // return the serialized size of the chunk header + all pages
+ return ChunkHeader.getSerializedSize(measurementId, pageBuffer.size())
+ + (long) pageBuffer.size();
+ }
+
+ public void sealCurrentPage() {
+ if (pageWriter != null && pageWriter.getPointNumber() > 0) {
+ writePageToPageBuffer();
+ }
+ }
+
+ public void clearPageWriter() {
+ pageWriter = null;
+ }
+
+ public int getNumOfPages() {
+ return numOfPages;
+ }
+
+ public TSDataType getDataType() {
+ return TSDataType.VECTOR;
+ }
+
+ /**
+ * write the page to specified IOWriter.
+ *
+ * @param writer the specified IOWriter
+ * @throws IOException exception in IO
+ */
+ public void writeAllPagesOfChunkToTsFile(TsFileIOWriter writer) throws IOException {
+ if (statistics.getCount() == 0) {
+ return;
+ }
+
+ // start to write this column chunk
+ writer.startFlushChunk(
+ measurementId,
+ compressionType,
+ TSDataType.VECTOR,
+ encodingType,
+ statistics,
+ pageBuffer.size(),
+ numOfPages,
+ 0x80);
+
+ long dataOffset = writer.getPos();
+
+ // write all pages of this column
+ writer.writeBytesToStream(pageBuffer);
+
+ int dataSize = (int) (writer.getPos() - dataOffset);
+ if (dataSize != pageBuffer.size()) {
+ throw new IOException(
+ "Bytes written is inconsistent with the size of data: "
+ + dataSize
+ + " !="
+ + " "
+ + pageBuffer.size());
+ }
+
+ writer.endCurrentChunk();
+ }
+
+ /** only used for test */
+ public PublicBAOS getPageBuffer() {
+ return pageBuffer;
+ }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ValueChunkWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ValueChunkWriter.java
new file mode 100644
index 0000000..17e137d
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ValueChunkWriter.java
@@ -0,0 +1,252 @@
+/*
+ * 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.tsfile.write.chunk;
+
+import org.apache.iotdb.tsfile.compress.ICompressor;
+import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
+import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+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.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.write.page.ValuePageWriter;
+import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+public class ValueChunkWriter {
+
+ private static final Logger logger = LoggerFactory.getLogger(ValueChunkWriter.class);
+
+ private final String measurementId;
+
+ private final TSEncoding encodingType;
+
+ private final TSDataType dataType;
+
+ private final CompressionType compressionType;
+
+ /** all pages of this chunk. */
+ private final PublicBAOS pageBuffer;
+
+ private int numOfPages;
+
+ /** write data into current page */
+ private ValuePageWriter pageWriter;
+
+ /** statistic of this chunk. */
+ private Statistics<?> statistics;
+
+ /** first page info */
+ private int sizeWithoutStatistic;
+
+ private Statistics<?> firstPageStatistics;
+
+ public ValueChunkWriter(
+ String measurementId,
+ CompressionType compressionType,
+ TSDataType dataType,
+ TSEncoding encodingType,
+ Encoder valueEncoder) {
+ this.measurementId = measurementId;
+ this.encodingType = encodingType;
+ this.dataType = dataType;
+ this.compressionType = compressionType;
+ this.pageBuffer = new PublicBAOS();
+
+ // init statistics for this chunk and page
+ this.statistics = Statistics.getStatsByType(dataType);
+
+ this.pageWriter =
+ new ValuePageWriter(valueEncoder, ICompressor.getCompressor(compressionType), dataType);
+ }
+
+ public void write(long time, long value, boolean isNull) {
+ pageWriter.write(time, value, isNull);
+ }
+
+ public void write(long time, int value, boolean isNull) {
+ pageWriter.write(time, value, isNull);
+ }
+
+ public void write(long time, boolean value, boolean isNull) {
+ pageWriter.write(time, value, isNull);
+ }
+
+ public void write(long time, float value, boolean isNull) {
+ pageWriter.write(time, value, isNull);
+ }
+
+ public void write(long time, double value, boolean isNull) {
+ pageWriter.write(time, value, isNull);
+ }
+
+ public void write(long time, Binary value, boolean isNull) {
+ pageWriter.write(time, value, isNull);
+ }
+
+ public void write(long[] timestamps, int[] values, int batchSize) {
+ pageWriter.write(timestamps, values, batchSize);
+ }
+
+ public void write(long[] timestamps, long[] values, int batchSize) {
+ pageWriter.write(timestamps, values, batchSize);
+ }
+
+ public void write(long[] timestamps, boolean[] values, int batchSize) {
+ pageWriter.write(timestamps, values, batchSize);
+ }
+
+ public void write(long[] timestamps, float[] values, int batchSize) {
+ pageWriter.write(timestamps, values, batchSize);
+ }
+
+ public void write(long[] timestamps, double[] values, int batchSize) {
+ pageWriter.write(timestamps, values, batchSize);
+ }
+
+ public void write(long[] timestamps, Binary[] values, int batchSize) {
+ pageWriter.write(timestamps, values, batchSize);
+ }
+
+ public void writePageToPageBuffer() {
+ try {
+ if (numOfPages == 0) { // record the firstPageStatistics
+ this.firstPageStatistics = pageWriter.getStatistics();
+ this.sizeWithoutStatistic = pageWriter.writePageHeaderAndDataIntoBuff(pageBuffer, true);
+ } else if (numOfPages == 1) { // put the firstPageStatistics into pageBuffer
+ byte[] b = pageBuffer.toByteArray();
+ pageBuffer.reset();
+ pageBuffer.write(b, 0, this.sizeWithoutStatistic);
+ firstPageStatistics.serialize(pageBuffer);
+ pageBuffer.write(b, this.sizeWithoutStatistic, b.length - this.sizeWithoutStatistic);
+ pageWriter.writePageHeaderAndDataIntoBuff(pageBuffer, false);
+ firstPageStatistics = null;
+ } else {
+ pageWriter.writePageHeaderAndDataIntoBuff(pageBuffer, false);
+ }
+
+ // update statistics of this chunk
+ numOfPages++;
+ this.statistics.mergeStatistics(pageWriter.getStatistics());
+ } catch (IOException e) {
+ logger.error("meet error in pageWriter.writePageHeaderAndDataIntoBuff,ignore this page:", e);
+ } finally {
+ // clear start time stamp for next initializing
+ pageWriter.reset(dataType);
+ }
+ }
+
+ public void writeToFileWriter(TsFileIOWriter tsfileWriter) throws IOException {
+ sealCurrentPage();
+ writeAllPagesOfChunkToTsFile(tsfileWriter);
+
+ // reinit this chunk writer
+ pageBuffer.reset();
+ numOfPages = 0;
+ firstPageStatistics = null;
+ this.statistics = Statistics.getStatsByType(dataType);
+ }
+
+ public long estimateMaxSeriesMemSize() {
+ return pageBuffer.size()
+ + pageWriter.estimateMaxMemSize()
+ + PageHeader.estimateMaxPageHeaderSizeWithoutStatistics()
+ + pageWriter.getStatistics().getSerializedSize();
+ }
+
+ public long getCurrentChunkSize() {
+ if (pageBuffer.size() == 0) {
+ return 0;
+ }
+ // return the serialized size of the chunk header + all pages
+ return ChunkHeader.getSerializedSize(measurementId, pageBuffer.size())
+ + (long) pageBuffer.size();
+ }
+
+ public void sealCurrentPage() {
+ // if the page contains no points, we still need to serialize it
+ if (pageWriter != null && pageWriter.getSize() != 0) {
+ writePageToPageBuffer();
+ }
+ }
+
+ public void clearPageWriter() {
+ pageWriter = null;
+ }
+
+ public int getNumOfPages() {
+ return numOfPages;
+ }
+
+ public TSDataType getDataType() {
+ return dataType;
+ }
+
+ /**
+ * write the page to specified IOWriter.
+ *
+ * @param writer the specified IOWriter
+ * @throws IOException exception in IO
+ */
+ public void writeAllPagesOfChunkToTsFile(TsFileIOWriter writer) throws IOException {
+ if (statistics.getCount() == 0) {
+ return;
+ }
+
+ // start to write this column chunk
+ writer.startFlushChunk(
+ measurementId,
+ compressionType,
+ dataType,
+ encodingType,
+ statistics,
+ pageBuffer.size(),
+ numOfPages,
+ 0x40);
+
+ long dataOffset = writer.getPos();
+
+ // write all pages of this column
+ writer.writeBytesToStream(pageBuffer);
+
+ int dataSize = (int) (writer.getPos() - dataOffset);
+ if (dataSize != pageBuffer.size()) {
+ throw new IOException(
+ "Bytes written is inconsistent with the size of data: "
+ + dataSize
+ + " !="
+ + " "
+ + pageBuffer.size());
+ }
+
+ writer.endCurrentChunk();
+ }
+
+ /** only used for test */
+ public PublicBAOS getPageBuffer() {
+ return pageBuffer;
+ }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/VectorChunkWriterImpl.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/VectorChunkWriterImpl.java
new file mode 100644
index 0000000..8f1e907
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/VectorChunkWriterImpl.java
@@ -0,0 +1,202 @@
+/*
+ * 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.tsfile.write.chunk;
+
+import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class VectorChunkWriterImpl implements IChunkWriter {
+
+ private final TimeChunkWriter timeChunkWriter;
+ private final List<ValueChunkWriter> valueChunkWriterList;
+ private int valueIndex;
+
+ /** @param schema schema of this measurement */
+ public VectorChunkWriterImpl(IMeasurementSchema schema) {
+ timeChunkWriter =
+ new TimeChunkWriter(
+ schema.getMeasurementId(),
+ schema.getCompressor(),
+ schema.getTimeTSEncoding(),
+ schema.getTimeEncoder());
+
+ List<String> valueMeasurementIdList = schema.getValueMeasurementIdList();
+ List<TSDataType> valueTSDataTypeList = schema.getValueTSDataTypeList();
+ List<TSEncoding> valueTSEncodingList = schema.getValueTSEncodingList();
+ List<Encoder> valueEncoderList = schema.getValueEncoderList();
+
+ valueChunkWriterList = new ArrayList<>(valueMeasurementIdList.size());
+ for (int i = 0; i < valueMeasurementIdList.size(); i++) {
+ valueChunkWriterList.add(
+ new ValueChunkWriter(
+ valueMeasurementIdList.get(i),
+ schema.getCompressor(),
+ valueTSDataTypeList.get(i),
+ valueTSEncodingList.get(i),
+ valueEncoderList.get(i)));
+ }
+
+ this.valueIndex = 0;
+ }
+
+ @Override
+ public void write(long time, int value, boolean isNull) {
+ valueChunkWriterList.get(valueIndex++).write(time, value, isNull);
+ }
+
+ @Override
+ public void write(long time, long value, boolean isNull) {
+ valueChunkWriterList.get(valueIndex++).write(time, value, isNull);
+ }
+
+ @Override
+ public void write(long time, boolean value, boolean isNull) {
+ valueChunkWriterList.get(valueIndex++).write(time, value, isNull);
+ }
+
+ @Override
+ public void write(long time, float value, boolean isNull) {
+ valueChunkWriterList.get(valueIndex++).write(time, value, isNull);
+ }
+
+ @Override
+ public void write(long time, double value, boolean isNull) {
+ valueChunkWriterList.get(valueIndex++).write(time, value, isNull);
+ }
+
+ @Override
+ public void write(long time, Binary value, boolean isNull) {
+ valueChunkWriterList.get(valueIndex++).write(time, value, isNull);
+ }
+
+ @Override
+ public void write(long time) {
+ valueIndex = 0;
+ timeChunkWriter.write(time);
+ if (checkPageSizeAndMayOpenANewPage()) {
+ writePageToPageBuffer();
+ }
+ }
+
+ // TODO tsfile write interface
+ @Override
+ public void write(long[] timestamps, int[] values, int batchSize) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void write(long[] timestamps, long[] values, int batchSize) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void write(long[] timestamps, boolean[] values, int batchSize) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void write(long[] timestamps, float[] values, int batchSize) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void write(long[] timestamps, double[] values, int batchSize) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void write(long[] timestamps, Binary[] values, int batchSize) {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * check occupied memory size, if it exceeds the PageSize threshold, construct a page and put it
+ * to pageBuffer
+ */
+ private boolean checkPageSizeAndMayOpenANewPage() {
+ return timeChunkWriter.checkPageSizeAndMayOpenANewPage();
+ }
+
+ private void writePageToPageBuffer() {
+ timeChunkWriter.writePageToPageBuffer();
+ for (ValueChunkWriter valueChunkWriter : valueChunkWriterList) {
+ valueChunkWriter.writePageToPageBuffer();
+ }
+ }
+
+ @Override
+ public void writeToFileWriter(TsFileIOWriter tsfileWriter) throws IOException {
+ timeChunkWriter.writeToFileWriter(tsfileWriter);
+ for (ValueChunkWriter valueChunkWriter : valueChunkWriterList) {
+ valueChunkWriter.writeToFileWriter(tsfileWriter);
+ }
+ }
+
+ @Override
+ public long estimateMaxSeriesMemSize() {
+ long estimateMaxSeriesMemSize = timeChunkWriter.estimateMaxSeriesMemSize();
+ for (ValueChunkWriter valueChunkWriter : valueChunkWriterList) {
+ estimateMaxSeriesMemSize += valueChunkWriter.estimateMaxSeriesMemSize();
+ }
+ return estimateMaxSeriesMemSize;
+ }
+
+ @Override
+ public long getCurrentChunkSize() {
+ long currentChunkSize = timeChunkWriter.getCurrentChunkSize();
+ for (ValueChunkWriter valueChunkWriter : valueChunkWriterList) {
+ currentChunkSize += valueChunkWriter.getCurrentChunkSize();
+ }
+ return currentChunkSize;
+ }
+
+ @Override
+ public void sealCurrentPage() {
+ timeChunkWriter.sealCurrentPage();
+ for (ValueChunkWriter valueChunkWriter : valueChunkWriterList) {
+ valueChunkWriter.sealCurrentPage();
+ }
+ }
+
+ @Override
+ public void clearPageWriter() {
+ timeChunkWriter.clearPageWriter();
+ for (ValueChunkWriter valueChunkWriter : valueChunkWriterList) {
+ valueChunkWriter.clearPageWriter();
+ }
+ }
+
+ @Override
+ public int getNumOfPages() {
+ return timeChunkWriter.getNumOfPages();
+ }
+
+ @Override
+ public TSDataType getDataType() {
+ return TSDataType.VECTOR;
+ }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/page/PageWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/page/PageWriter.java
index 8467d15..5fa9977 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/page/PageWriter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/page/PageWriter.java
@@ -26,7 +26,7 @@
import org.apache.iotdb.tsfile.utils.Binary;
import org.apache.iotdb.tsfile.utils.PublicBAOS;
import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -63,7 +63,7 @@
this(null, null);
}
- public PageWriter(MeasurementSchema measurementSchema) {
+ public PageWriter(IMeasurementSchema measurementSchema) {
this(measurementSchema.getTimeEncoder(), measurementSchema.getValueEncoder());
this.statistics = Statistics.getStatsByType(measurementSchema.getType());
this.compressor = ICompressor.getCompressor(measurementSchema.getCompressor());
@@ -263,7 +263,7 @@
}
/** reset this page */
- public void reset(MeasurementSchema measurementSchema) {
+ public void reset(IMeasurementSchema measurementSchema) {
timeOut.reset();
valueOut.reset();
statistics = Statistics.getStatsByType(measurementSchema.getType());
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/page/TimePageWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/page/TimePageWriter.java
new file mode 100644
index 0000000..5223d18
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/page/TimePageWriter.java
@@ -0,0 +1,177 @@
+/*
+ * 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.tsfile.write.page;
+
+import org.apache.iotdb.tsfile.compress.ICompressor;
+import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.TimeStatistics;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+
+/**
+ * This writer is used to write time into a page. It consists of a time encoder and respective
+ * OutputStream.
+ */
+public class TimePageWriter {
+
+ private static final Logger logger = LoggerFactory.getLogger(TimePageWriter.class);
+
+ private final ICompressor compressor;
+
+ // time
+ private Encoder timeEncoder;
+ private final PublicBAOS timeOut;
+
+ /**
+ * statistic of current page. It will be reset after calling {@code
+ * writePageHeaderAndDataIntoBuff()}
+ */
+ private TimeStatistics statistics;
+
+ public TimePageWriter(Encoder timeEncoder, ICompressor compressor) {
+ this.timeOut = new PublicBAOS();
+ this.timeEncoder = timeEncoder;
+ this.statistics = new TimeStatistics();
+ this.compressor = compressor;
+ }
+
+ /** write a time into encoder */
+ public void write(long time) {
+ timeEncoder.encode(time, timeOut);
+ statistics.update(time);
+ }
+
+ /** write time series into encoder */
+ public void write(long[] timestamps, int batchSize) {
+ for (int i = 0; i < batchSize; i++) {
+ timeEncoder.encode(timestamps[i], timeOut);
+ }
+ statistics.update(timestamps, batchSize);
+ }
+
+ /** flush all data remained in encoders. */
+ private void prepareEndWriteOnePage() throws IOException {
+ timeEncoder.flush(timeOut);
+ }
+
+ /**
+ * getUncompressedBytes return data what it has been written in form of <code>
+ * size of time list, time list, value list</code>
+ *
+ * @return a new readable ByteBuffer whose position is 0.
+ */
+ public ByteBuffer getUncompressedBytes() throws IOException {
+ prepareEndWriteOnePage();
+ ByteBuffer buffer = ByteBuffer.allocate(timeOut.size());
+ buffer.put(timeOut.getBuf(), 0, timeOut.size());
+ buffer.flip();
+ return buffer;
+ }
+
+ /** write the page header and data into the PageWriter's output stream. */
+ public int writePageHeaderAndDataIntoBuff(PublicBAOS pageBuffer, boolean first)
+ throws IOException {
+ if (statistics.getCount() == 0) {
+ return 0;
+ }
+
+ ByteBuffer pageData = getUncompressedBytes();
+ int uncompressedSize = pageData.remaining();
+ int compressedSize;
+ byte[] compressedBytes = null;
+
+ if (compressor.getType().equals(CompressionType.UNCOMPRESSED)) {
+ compressedSize = uncompressedSize;
+ } else {
+ compressedBytes = new byte[compressor.getMaxBytesForCompression(uncompressedSize)];
+ // data is never a directByteBuffer now, so we can use data.array()
+ compressedSize =
+ compressor.compress(
+ pageData.array(), pageData.position(), uncompressedSize, compressedBytes);
+ }
+
+ // write the page header to IOWriter
+ int sizeWithoutStatistic = 0;
+ if (first) {
+ sizeWithoutStatistic +=
+ ReadWriteForEncodingUtils.writeUnsignedVarInt(uncompressedSize, pageBuffer);
+ sizeWithoutStatistic +=
+ ReadWriteForEncodingUtils.writeUnsignedVarInt(compressedSize, pageBuffer);
+ } else {
+ ReadWriteForEncodingUtils.writeUnsignedVarInt(uncompressedSize, pageBuffer);
+ ReadWriteForEncodingUtils.writeUnsignedVarInt(compressedSize, pageBuffer);
+ statistics.serialize(pageBuffer);
+ }
+
+ // write page content to temp PBAOS
+ logger.trace(
+ "start to flush a time page data into buffer, buffer position {} ", pageBuffer.size());
+ if (compressor.getType().equals(CompressionType.UNCOMPRESSED)) {
+ try (WritableByteChannel channel = Channels.newChannel(pageBuffer)) {
+ channel.write(pageData);
+ }
+ } else {
+ pageBuffer.write(compressedBytes, 0, compressedSize);
+ }
+ logger.trace(
+ "finish flushing a time page data into buffer, buffer position {} ", pageBuffer.size());
+ return sizeWithoutStatistic;
+ }
+
+ /**
+ * calculate max possible memory size it occupies, including time outputStream and value
+ * outputStream, because size outputStream is never used until flushing.
+ *
+ * @return allocated size in time, value and outputStream
+ */
+ public long estimateMaxMemSize() {
+ return timeOut.size() + timeEncoder.getMaxByteSize();
+ }
+
+ /** reset this page */
+ public void reset() {
+ timeOut.reset();
+ statistics = new TimeStatistics();
+ }
+
+ public void setTimeEncoder(Encoder encoder) {
+ this.timeEncoder = encoder;
+ }
+
+ public void initStatistics() {
+ statistics = new TimeStatistics();
+ }
+
+ public long getPointNumber() {
+ return statistics.getCount();
+ }
+
+ public TimeStatistics getStatistics() {
+ return statistics;
+ }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/page/ValuePageWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/page/ValuePageWriter.java
new file mode 100644
index 0000000..0fbfdbd
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/page/ValuePageWriter.java
@@ -0,0 +1,307 @@
+/*
+ * 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.tsfile.write.page;
+
+import org.apache.iotdb.tsfile.compress.ICompressor;
+import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
+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.statistics.Statistics;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+
+/**
+ * This writer is used to write value into a page. It consists of a value encoder and respective
+ * OutputStream.
+ */
+public class ValuePageWriter {
+ private static final Logger logger = LoggerFactory.getLogger(ValuePageWriter.class);
+
+ private final ICompressor compressor;
+
+ // value
+ private Encoder valueEncoder;
+ private final PublicBAOS valueOut;
+
+ /**
+ * statistic of current page. It will be reset after calling {@code
+ * writePageHeaderAndDataIntoBuff()}
+ */
+ private Statistics<?> statistics;
+
+ private byte bitmap;
+
+ private int size;
+
+ private final PublicBAOS bitmapOut;
+
+ private static final int MASK = 1 << 7;
+
+ public ValuePageWriter(Encoder valueEncoder, ICompressor compressor, TSDataType dataType) {
+ this.valueOut = new PublicBAOS();
+ this.bitmap = 0;
+ this.size = 0;
+ this.bitmapOut = new PublicBAOS();
+ this.valueEncoder = valueEncoder;
+ this.statistics = Statistics.getStatsByType(dataType);
+ this.compressor = compressor;
+ }
+
+ /** write a time value pair into encoder */
+ public void write(long time, boolean value, boolean isNull) {
+ setBit(isNull);
+ if (!isNull) {
+ valueEncoder.encode(value, valueOut);
+ statistics.update(time, value);
+ }
+ }
+
+ /** write a time value pair into encoder */
+ public void write(long time, short value, boolean isNull) {
+ setBit(isNull);
+ if (!isNull) {
+ valueEncoder.encode(value, valueOut);
+ statistics.update(time, value);
+ }
+ }
+
+ /** write a time value pair into encoder */
+ public void write(long time, int value, boolean isNull) {
+ setBit(isNull);
+ if (!isNull) {
+ valueEncoder.encode(value, valueOut);
+ statistics.update(time, value);
+ }
+ }
+
+ /** write a time value pair into encoder */
+ public void write(long time, long value, boolean isNull) {
+ setBit(isNull);
+ if (!isNull) {
+ valueEncoder.encode(value, valueOut);
+ statistics.update(time, value);
+ }
+ }
+
+ /** write a time value pair into encoder */
+ public void write(long time, float value, boolean isNull) {
+ setBit(isNull);
+ if (!isNull) {
+ valueEncoder.encode(value, valueOut);
+ statistics.update(time, value);
+ }
+ }
+
+ /** write a time value pair into encoder */
+ public void write(long time, double value, boolean isNull) {
+ setBit(isNull);
+ if (!isNull) {
+ valueEncoder.encode(value, valueOut);
+ statistics.update(time, value);
+ }
+ }
+
+ /** write a time value pair into encoder */
+ public void write(long time, Binary value, boolean isNull) {
+ setBit(isNull);
+ if (!isNull) {
+ valueEncoder.encode(value, valueOut);
+ statistics.update(time, value);
+ }
+ }
+
+ private void setBit(boolean isNull) {
+ if (!isNull) {
+ bitmap |= (MASK >>> (size % 8));
+ }
+ size++;
+ if (size % 8 == 0) {
+ bitmapOut.write(bitmap);
+ bitmap = 0;
+ }
+ }
+
+ /** write time series into encoder */
+ public void write(long[] timestamps, boolean[] values, int batchSize) {
+ for (int i = 0; i < batchSize; i++) {
+ valueEncoder.encode(values[i], valueOut);
+ }
+ statistics.update(timestamps, values, batchSize);
+ }
+
+ /** write time series into encoder */
+ public void write(long[] timestamps, int[] values, int batchSize) {
+ for (int i = 0; i < batchSize; i++) {
+ valueEncoder.encode(values[i], valueOut);
+ }
+ statistics.update(timestamps, values, batchSize);
+ }
+
+ /** write time series into encoder */
+ public void write(long[] timestamps, long[] values, int batchSize) {
+ for (int i = 0; i < batchSize; i++) {
+ valueEncoder.encode(values[i], valueOut);
+ }
+ statistics.update(timestamps, values, batchSize);
+ }
+
+ /** write time series into encoder */
+ public void write(long[] timestamps, float[] values, int batchSize) {
+ for (int i = 0; i < batchSize; i++) {
+ valueEncoder.encode(values[i], valueOut);
+ }
+ statistics.update(timestamps, values, batchSize);
+ }
+
+ /** write time series into encoder */
+ public void write(long[] timestamps, double[] values, int batchSize) {
+ for (int i = 0; i < batchSize; i++) {
+ valueEncoder.encode(values[i], valueOut);
+ }
+ statistics.update(timestamps, values, batchSize);
+ }
+
+ /** write time series into encoder */
+ public void write(long[] timestamps, Binary[] values, int batchSize) {
+ for (int i = 0; i < batchSize; i++) {
+ valueEncoder.encode(values[i], valueOut);
+ }
+ statistics.update(timestamps, values, batchSize);
+ }
+
+ /** flush all data remained in encoders. */
+ private void prepareEndWriteOnePage() throws IOException {
+ valueEncoder.flush(valueOut);
+ if (size % 8 != 0) {
+ bitmapOut.write(bitmap);
+ }
+ }
+
+ /**
+ * getUncompressedBytes return data what it has been written in form of <code>
+ * size of time list, time list, value list</code>
+ *
+ * @return a new readable ByteBuffer whose position is 0.
+ */
+ public ByteBuffer getUncompressedBytes() throws IOException {
+ prepareEndWriteOnePage();
+ ByteBuffer buffer = ByteBuffer.allocate(Integer.BYTES + bitmapOut.size() + valueOut.size());
+ buffer.putInt(size);
+ buffer.put(bitmapOut.getBuf(), 0, bitmapOut.size());
+ buffer.put(valueOut.getBuf(), 0, valueOut.size());
+ buffer.flip();
+ return buffer;
+ }
+
+ /** write the page header and data into the PageWriter's output stream. */
+ public int writePageHeaderAndDataIntoBuff(PublicBAOS pageBuffer, boolean first)
+ throws IOException {
+ if (size == 0) {
+ return 0;
+ }
+
+ ByteBuffer pageData = getUncompressedBytes();
+ int uncompressedSize = pageData.remaining();
+ int compressedSize;
+ byte[] compressedBytes = null;
+
+ if (compressor.getType().equals(CompressionType.UNCOMPRESSED)) {
+ compressedSize = uncompressedSize;
+ } else {
+ compressedBytes = new byte[compressor.getMaxBytesForCompression(uncompressedSize)];
+ // data is never a directByteBuffer now, so we can use data.array()
+ compressedSize =
+ compressor.compress(
+ pageData.array(), pageData.position(), uncompressedSize, compressedBytes);
+ }
+
+ // write the page header to IOWriter
+ int sizeWithoutStatistic = 0;
+ if (first) {
+ sizeWithoutStatistic +=
+ ReadWriteForEncodingUtils.writeUnsignedVarInt(uncompressedSize, pageBuffer);
+ sizeWithoutStatistic +=
+ ReadWriteForEncodingUtils.writeUnsignedVarInt(compressedSize, pageBuffer);
+ } else {
+ ReadWriteForEncodingUtils.writeUnsignedVarInt(uncompressedSize, pageBuffer);
+ ReadWriteForEncodingUtils.writeUnsignedVarInt(compressedSize, pageBuffer);
+ statistics.serialize(pageBuffer);
+ }
+
+ // write page content to temp PBAOS
+ logger.trace("start to flush a page data into buffer, buffer position {} ", pageBuffer.size());
+ if (compressor.getType().equals(CompressionType.UNCOMPRESSED)) {
+ try (WritableByteChannel channel = Channels.newChannel(pageBuffer)) {
+ channel.write(pageData);
+ }
+ } else {
+ pageBuffer.write(compressedBytes, 0, compressedSize);
+ }
+ logger.trace("start to flush a page data into buffer, buffer position {} ", pageBuffer.size());
+ return sizeWithoutStatistic;
+ }
+
+ /**
+ * calculate max possible memory size it occupies, including time outputStream and value
+ * outputStream, because size outputStream is never used until flushing.
+ *
+ * @return allocated size in time, value and outputStream
+ */
+ public long estimateMaxMemSize() {
+ return Integer.BYTES + bitmapOut.size() + 1 + valueOut.size() + valueEncoder.getMaxByteSize();
+ }
+
+ /** reset this page */
+ public void reset(TSDataType dataType) {
+ bitmapOut.reset();
+ size = 0;
+ bitmap = 0;
+ valueOut.reset();
+ statistics = Statistics.getStatsByType(dataType);
+ }
+
+ public void setValueEncoder(Encoder encoder) {
+ this.valueEncoder = encoder;
+ }
+
+ public void initStatistics(TSDataType dataType) {
+ statistics = Statistics.getStatsByType(dataType);
+ }
+
+ public long getPointNumber() {
+ return statistics.getCount();
+ }
+
+ public Statistics<?> getStatistics() {
+ return statistics;
+ }
+
+ public int getSize() {
+ return size;
+ }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/Tablet.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/Tablet.java
index 461f046..5f47c79 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/Tablet.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/Tablet.java
@@ -21,8 +21,12 @@
import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.BitMap;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
+import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -46,15 +50,20 @@
public String deviceId;
/** the list of measurement schemas for creating the tablet */
- private List<MeasurementSchema> schemas;
+ private List<IMeasurementSchema> schemas;
/** measurementId->indexOf(measurementSchema) */
- private Map<String, Integer> measurementIndex;
+ private Map<String, Integer> measurementIndexInSchema;
+
+ /** measurementId->indexOf(values) */
+ private Map<String, Integer> measurementIndexInValues;
/** timestamps in this tablet */
public long[] timestamps;
/** each object is a primitive type array, which represents values of one measurement */
public Object[] values;
+ /** each bitmap represents the existence of each value in the current column. */
+ public BitMap[] bitMaps;
/** the number of rows to include in this tablet */
public int rowSize;
/** the maximum number of rows for this tablet */
@@ -68,7 +77,7 @@
* @param schemas the list of measurement schemas for creating the tablet, only measurementId and
* type take effects
*/
- public Tablet(String deviceId, List<MeasurementSchema> schemas) {
+ public Tablet(String deviceId, List<IMeasurementSchema> schemas) {
this(deviceId, schemas, DEFAULT_SIZE);
}
@@ -81,14 +90,28 @@
* and type take effects
* @param maxRowNumber the maximum number of rows for this tablet
*/
- public Tablet(String deviceId, List<MeasurementSchema> schemas, int maxRowNumber) {
+ public Tablet(String deviceId, List<IMeasurementSchema> schemas, int maxRowNumber) {
this.deviceId = deviceId;
- this.schemas = schemas;
+ this.schemas = new ArrayList<>(schemas);
this.maxRowNumber = maxRowNumber;
- measurementIndex = new HashMap<>();
+ measurementIndexInSchema = new HashMap<>();
+ measurementIndexInValues = new HashMap<>();
- for (int i = 0; i < schemas.size(); i++) {
- measurementIndex.put(schemas.get(i).getMeasurementId(), i);
+ int indexInValues = 0;
+ int indexInSchema = 0;
+ for (IMeasurementSchema schema : schemas) {
+ if (schema.getType() == TSDataType.VECTOR) {
+ for (String measurementId : schema.getValueMeasurementIdList()) {
+ measurementIndexInValues.put(measurementId, indexInValues);
+ measurementIndexInSchema.put(measurementId, indexInSchema);
+ indexInValues++;
+ }
+ } else {
+ measurementIndexInValues.put(schema.getMeasurementId(), indexInValues);
+ measurementIndexInSchema.put(schema.getMeasurementId(), indexInSchema);
+ indexInValues++;
+ }
+ indexInSchema++;
}
createColumns();
@@ -96,58 +119,86 @@
reset();
}
+ public void setDeviceId(String deviceId) {
+ this.deviceId = deviceId;
+ }
+
public void addTimestamp(int rowIndex, long timestamp) {
timestamps[rowIndex] = timestamp;
}
+ // (s1, s2) s3
public void addValue(String measurementId, int rowIndex, Object value) {
- int indexOfValue = measurementIndex.get(measurementId);
- MeasurementSchema measurementSchema = schemas.get(indexOfValue);
+ int indexOfValues = measurementIndexInValues.get(measurementId);
+ int indexOfSchema = measurementIndexInSchema.get(measurementId);
+ IMeasurementSchema measurementSchema = schemas.get(indexOfSchema);
- switch (measurementSchema.getType()) {
+ if (measurementSchema.getType().equals(TSDataType.VECTOR)) {
+ int indexInVector = measurementSchema.getMeasurementIdColumnIndex(measurementId);
+ TSDataType dataType = measurementSchema.getValueTSDataTypeList().get(indexInVector);
+ addValueOfDataType(dataType, rowIndex, indexOfValues, value);
+ } else {
+ addValueOfDataType(measurementSchema.getType(), rowIndex, indexOfValues, value);
+ }
+ }
+
+ private void addValueOfDataType(
+ TSDataType dataType, int rowIndex, int indexOfValue, Object value) {
+
+ if (value == null) {
+ // init the bitMap to mark null value
+ if (bitMaps == null) {
+ bitMaps = new BitMap[values.length];
+ }
+ if (bitMaps[indexOfValue] == null) {
+ bitMaps[indexOfValue] = new BitMap(maxRowNumber);
+ }
+ // mark the null value position
+ bitMaps[indexOfValue].mark(rowIndex);
+ }
+ switch (dataType) {
case TEXT:
{
Binary[] sensor = (Binary[]) values[indexOfValue];
- sensor[rowIndex] = (Binary) value;
+ sensor[rowIndex] = value != null ? (Binary) value : Binary.EMPTY_VALUE;
break;
}
case FLOAT:
{
float[] sensor = (float[]) values[indexOfValue];
- sensor[rowIndex] = (float) value;
+ sensor[rowIndex] = value != null ? (float) value : Float.MIN_VALUE;
break;
}
case INT32:
{
int[] sensor = (int[]) values[indexOfValue];
- sensor[rowIndex] = (int) value;
+ sensor[rowIndex] = value != null ? (int) value : Integer.MIN_VALUE;
break;
}
case INT64:
{
long[] sensor = (long[]) values[indexOfValue];
- sensor[rowIndex] = (long) value;
+ sensor[rowIndex] = value != null ? (long) value : Long.MIN_VALUE;
break;
}
case DOUBLE:
{
double[] sensor = (double[]) values[indexOfValue];
- sensor[rowIndex] = (double) value;
+ sensor[rowIndex] = value != null ? (double) value : Double.MIN_VALUE;
break;
}
case BOOLEAN:
{
boolean[] sensor = (boolean[]) values[indexOfValue];
- sensor[rowIndex] = (boolean) value;
+ sensor[rowIndex] = value != null && (boolean) value;
break;
}
default:
- throw new UnSupportedDataTypeException(
- String.format(NOT_SUPPORT_DATATYPE, measurementSchema.getType()));
+ throw new UnSupportedDataTypeException(String.format(NOT_SUPPORT_DATATYPE, dataType));
}
}
- public List<MeasurementSchema> getSchemas() {
+ public List<IMeasurementSchema> getSchemas() {
return schemas;
}
@@ -164,33 +215,66 @@
private void createColumns() {
// create timestamp column
timestamps = new long[maxRowNumber];
- values = new Object[schemas.size()];
- // create value columns
- for (int i = 0; i < schemas.size(); i++) {
- TSDataType dataType = schemas.get(i).getType();
- switch (dataType) {
- case INT32:
- values[i] = new int[maxRowNumber];
- break;
- case INT64:
- values[i] = new long[maxRowNumber];
- break;
- case FLOAT:
- values[i] = new float[maxRowNumber];
- break;
- case DOUBLE:
- values[i] = new double[maxRowNumber];
- break;
- case BOOLEAN:
- values[i] = new boolean[maxRowNumber];
- break;
- case TEXT:
- values[i] = new Binary[maxRowNumber];
- break;
- default:
- throw new UnSupportedDataTypeException(String.format(NOT_SUPPORT_DATATYPE, dataType));
+
+ // calculate total value column size
+ int valueColumnsSize = 0;
+ for (IMeasurementSchema schema : schemas) {
+ if (schema instanceof VectorMeasurementSchema) {
+ valueColumnsSize += schema.getValueMeasurementIdList().size();
+ } else {
+ valueColumnsSize++;
}
}
+
+ // value column
+ values = new Object[valueColumnsSize];
+ int columnIndex = 0;
+ for (IMeasurementSchema schema : schemas) {
+ TSDataType dataType = schema.getType();
+ if (dataType.equals(TSDataType.VECTOR)) {
+ columnIndex = buildVectorColumns((VectorMeasurementSchema) schema, columnIndex);
+ } else {
+ values[columnIndex] = createValueColumnOfDataType(dataType);
+ columnIndex++;
+ }
+ }
+ }
+
+ private int buildVectorColumns(VectorMeasurementSchema schema, int idx) {
+ for (int i = 0; i < schema.getValueMeasurementIdList().size(); i++) {
+ TSDataType dataType = schema.getValueTSDataTypeList().get(i);
+ values[idx] = createValueColumnOfDataType(dataType);
+ idx++;
+ }
+ return idx;
+ }
+
+ private Object createValueColumnOfDataType(TSDataType dataType) {
+
+ Object valueColumn;
+ switch (dataType) {
+ case INT32:
+ valueColumn = new int[maxRowNumber];
+ break;
+ case INT64:
+ valueColumn = new long[maxRowNumber];
+ break;
+ case FLOAT:
+ valueColumn = new float[maxRowNumber];
+ break;
+ case DOUBLE:
+ valueColumn = new double[maxRowNumber];
+ break;
+ case BOOLEAN:
+ valueColumn = new boolean[maxRowNumber];
+ break;
+ case TEXT:
+ valueColumn = new Binary[maxRowNumber];
+ break;
+ default:
+ throw new UnSupportedDataTypeException(String.format(NOT_SUPPORT_DATATYPE, dataType));
+ }
+ return valueColumn;
}
public int getTimeBytesSize() {
@@ -198,38 +282,59 @@
}
/** @return total bytes of values */
- public int getValueBytesSize() {
- /** total byte size that values occupies */
+ public int getTotalValueOccupation() {
int valueOccupation = 0;
+ int columnIndex = 0;
for (int i = 0; i < schemas.size(); i++) {
- switch (schemas.get(i).getType()) {
- case BOOLEAN:
- valueOccupation += rowSize;
- break;
- case INT32:
- valueOccupation += rowSize * 4;
- break;
- case INT64:
- valueOccupation += rowSize * 8;
- break;
- case FLOAT:
- valueOccupation += rowSize * 4;
- break;
- case DOUBLE:
- valueOccupation += rowSize * 8;
- break;
- case TEXT:
- valueOccupation += rowSize * 4;
- Binary[] binaries = (Binary[]) values[i];
- for (int rowIndex = 0; rowIndex < rowSize; rowIndex++) {
- valueOccupation += binaries[rowIndex].getLength();
- }
- break;
- default:
- throw new UnSupportedDataTypeException(
- String.format(NOT_SUPPORT_DATATYPE, schemas.get(i).getType()));
+ IMeasurementSchema schema = schemas.get(i);
+ if (schema instanceof MeasurementSchema) {
+ valueOccupation += calOccupationOfOneColumn(schema.getType(), columnIndex);
+ columnIndex++;
+ } else {
+ for (int j = 0; j < schema.getValueTSDataTypeList().size(); j++) {
+ TSDataType dataType = schema.getValueTSDataTypeList().get(j);
+ valueOccupation += calOccupationOfOneColumn(dataType, columnIndex);
+ columnIndex++;
+ }
}
}
+ // add bitmap size if the tablet has bitMaps
+ if (bitMaps != null) {
+ for (BitMap bitMap : bitMaps) {
+ // marker byte
+ valueOccupation++;
+ if (bitMap != null && !bitMap.isAllUnmarked()) {
+ valueOccupation += rowSize / Byte.SIZE + 1;
+ }
+ }
+ }
+ return valueOccupation;
+ }
+
+ private int calOccupationOfOneColumn(TSDataType dataType, int columnIndex) {
+ int valueOccupation = 0;
+ switch (dataType) {
+ case BOOLEAN:
+ valueOccupation += rowSize;
+ break;
+ case INT32:
+ case FLOAT:
+ valueOccupation += rowSize * 4;
+ break;
+ case INT64:
+ case DOUBLE:
+ valueOccupation += rowSize * 8;
+ break;
+ case TEXT:
+ valueOccupation += rowSize * 4;
+ Binary[] binaries = (Binary[]) values[columnIndex];
+ for (int rowIndex = 0; rowIndex < rowSize; rowIndex++) {
+ valueOccupation += binaries[rowIndex].getLength();
+ }
+ break;
+ default:
+ throw new UnSupportedDataTypeException(String.format(NOT_SUPPORT_DATATYPE, dataType));
+ }
return valueOccupation;
}
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/BooleanDataPoint.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/BooleanDataPoint.java
index b607a79..e8b4bee 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/BooleanDataPoint.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/BooleanDataPoint.java
@@ -47,7 +47,7 @@
LOG.warn("given IChunkWriter is null, do nothing and return");
return;
}
- writer.write(time, value);
+ writer.write(time, value, false);
}
@Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/DoubleDataPoint.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/DoubleDataPoint.java
index b988ed7..853313e 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/DoubleDataPoint.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/DoubleDataPoint.java
@@ -47,7 +47,7 @@
LOG.warn("given IChunkWriter is null, do nothing and return");
return;
}
- writer.write(time, value);
+ writer.write(time, value, false);
}
@Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/FloatDataPoint.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/FloatDataPoint.java
index 3d13f17..863be98 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/FloatDataPoint.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/FloatDataPoint.java
@@ -47,7 +47,7 @@
LOG.warn("given IChunkWriter is null, do nothing and return");
return;
}
- writer.write(time, value);
+ writer.write(time, value, false);
}
@Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/IntDataPoint.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/IntDataPoint.java
index a66378e..02e0d5c 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/IntDataPoint.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/IntDataPoint.java
@@ -47,7 +47,7 @@
LOG.warn("given IChunkWriter is null, do nothing and return");
return;
}
- writer.write(time, value);
+ writer.write(time, value, false);
}
@Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/LongDataPoint.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/LongDataPoint.java
index 4bc2e9f..8dce510 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/LongDataPoint.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/LongDataPoint.java
@@ -47,7 +47,7 @@
LOG.warn("given IChunkWriter is null, do nothing and return");
return;
}
- writer.write(time, value);
+ writer.write(time, value, false);
}
@Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/StringDataPoint.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/StringDataPoint.java
index 3a3918c..cf371bc 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/StringDataPoint.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/datapoint/StringDataPoint.java
@@ -48,7 +48,7 @@
LOG.warn("given IChunkWriter is null, do nothing and return");
return;
}
- writer.write(time, value);
+ writer.write(time, value, false);
}
@Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/IMeasurementSchema.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/IMeasurementSchema.java
new file mode 100644
index 0000000..fb3358d
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/IMeasurementSchema.java
@@ -0,0 +1,77 @@
+/*
+ * 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.tsfile.write.schema;
+
+import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
+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 java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+public interface IMeasurementSchema {
+
+ String getMeasurementId();
+
+ CompressionType getCompressor();
+
+ TSEncoding getEncodingType();
+
+ TSDataType getType();
+
+ void setType(TSDataType dataType);
+
+ TSEncoding getTimeTSEncoding();
+
+ Encoder getTimeEncoder();
+
+ Encoder getValueEncoder();
+
+ Map<String, String> getProps();
+
+ List<String> getValueMeasurementIdList();
+
+ List<TSDataType> getValueTSDataTypeList();
+
+ List<TSEncoding> getValueTSEncodingList();
+
+ List<Encoder> getValueEncoderList();
+
+ int getMeasurementIdColumnIndex(String measurementId);
+
+ int serializeTo(ByteBuffer buffer);
+
+ int serializeTo(OutputStream outputStream) throws IOException;
+
+ /*
+ 1. used in cluster module to avoid useless field transfer(such as props in MeasurementSchema)
+ 2. add a flag bit at the beginning to distinguish between MeasurementSchema(0) and VectorMeasurementSchema(1)
+ */
+ int partialSerializeTo(ByteBuffer buffer);
+
+ /*
+ 1. used in cluster module to avoid useless field transfer(such as props in MeasurementSchema)
+ 2. add a flag bit at the beginning to distinguish between MeasurementSchema(0) and VectorMeasurementSchema(1)
+ */
+ int partialSerializeTo(OutputStream outputStream) throws IOException;
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/MeasurementSchema.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/MeasurementSchema.java
index 34ee496..4778ca6 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/MeasurementSchema.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/MeasurementSchema.java
@@ -33,6 +33,7 @@
import java.io.Serializable;
import java.nio.ByteBuffer;
import java.util.HashMap;
+import java.util.List;
import java.util.Map;
import java.util.Objects;
@@ -42,14 +43,8 @@
* MeasurementSchema maintains respective TSEncodingBuilder; For TSDataType, only ENUM has
* TSDataTypeConverter up to now.
*/
-public class MeasurementSchema implements Comparable<MeasurementSchema>, Serializable {
-
- public static final MeasurementSchema TIME_SCHEMA =
- new MeasurementSchema(
- "time",
- TSDataType.INT64,
- TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()),
- TSFileDescriptor.getInstance().getConfig().getCompressor());
+public class MeasurementSchema
+ implements IMeasurementSchema, Comparable<MeasurementSchema>, Serializable {
private String measurementId;
private byte type;
@@ -170,6 +165,21 @@
return measurementSchema;
}
+ public static MeasurementSchema partialDeserializeFrom(ByteBuffer buffer) {
+ MeasurementSchema measurementSchema = new MeasurementSchema();
+
+ measurementSchema.measurementId = ReadWriteIOUtils.readString(buffer);
+
+ measurementSchema.type = ReadWriteIOUtils.readByte(buffer);
+
+ measurementSchema.encoding = ReadWriteIOUtils.readByte(buffer);
+
+ measurementSchema.compressor = ReadWriteIOUtils.readByte(buffer);
+
+ return measurementSchema;
+ }
+
+ @Override
public String getMeasurementId() {
return measurementId;
}
@@ -178,23 +188,32 @@
this.measurementId = measurementId;
}
+ @Override
public Map<String, String> getProps() {
return props;
}
+ @Override
public TSEncoding getEncodingType() {
return TSEncoding.deserialize(encoding);
}
+ @Override
public TSDataType getType() {
return TSDataType.deserialize(type);
}
+ @Override
+ public TSEncoding getTimeTSEncoding() {
+ return TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder());
+ }
+
public void setProps(Map<String, String> props) {
this.props = props;
}
/** function for getting time encoder. */
+ @Override
public Encoder getTimeEncoder() {
TSEncoding timeEncoding =
TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder());
@@ -203,6 +222,26 @@
return TSEncodingBuilder.getEncodingBuilder(timeEncoding).getEncoder(timeType);
}
+ @Override
+ public List<String> getValueMeasurementIdList() {
+ throw new UnsupportedOperationException("unsupported method for MeasurementSchema");
+ }
+
+ @Override
+ public List<TSDataType> getValueTSDataTypeList() {
+ throw new UnsupportedOperationException("unsupported method for MeasurementSchema");
+ }
+
+ @Override
+ public List<TSEncoding> getValueTSEncodingList() {
+ throw new UnsupportedOperationException("unsupported method for MeasurementSchema");
+ }
+
+ @Override
+ public List<Encoder> getValueEncoderList() {
+ throw new UnsupportedOperationException("unsupported method for MeasurementSchema");
+ }
+
/**
* get Encoder of value from encodingConverter by measurementID and data type.
*
@@ -218,11 +257,13 @@
return encodingConverter.getEncoder(TSDataType.deserialize(type));
}
+ @Override
public CompressionType getCompressor() {
return CompressionType.deserialize(compressor);
}
/** function for serializing data to output stream. */
+ @Override
public int serializeTo(OutputStream outputStream) throws IOException {
int byteLen = 0;
@@ -248,6 +289,7 @@
}
/** function for serializing data to byte buffer. */
+ @Override
public int serializeTo(ByteBuffer buffer) {
int byteLen = 0;
@@ -273,6 +315,32 @@
}
@Override
+ public int partialSerializeTo(OutputStream outputStream) throws IOException {
+ int byteLen = 0;
+
+ byteLen += ReadWriteIOUtils.write((byte) 0, outputStream);
+ byteLen += ReadWriteIOUtils.write(measurementId, outputStream);
+ byteLen += ReadWriteIOUtils.write(type, outputStream);
+ byteLen += ReadWriteIOUtils.write(encoding, outputStream);
+ byteLen += ReadWriteIOUtils.write(compressor, outputStream);
+
+ return byteLen;
+ }
+
+ @Override
+ public int partialSerializeTo(ByteBuffer buffer) {
+ int byteLen = 0;
+
+ byteLen += ReadWriteIOUtils.write((byte) 0, buffer);
+ byteLen += ReadWriteIOUtils.write(measurementId, buffer);
+ byteLen += ReadWriteIOUtils.write(type, buffer);
+ byteLen += ReadWriteIOUtils.write(encoding, buffer);
+ byteLen += ReadWriteIOUtils.write(compressor, buffer);
+
+ return byteLen;
+ }
+
+ @Override
public boolean equals(Object o) {
if (this == o) {
return true;
@@ -323,4 +391,9 @@
public void setType(TSDataType type) {
this.type = type.serialize();
}
+
+ @Override
+ public int getMeasurementIdColumnIndex(String measurementId) {
+ return 0;
+ }
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/Schema.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/Schema.java
index a8f19fe..cf45286 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/Schema.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/Schema.java
@@ -36,35 +36,36 @@
* Path (device + measurement) -> measurementSchema By default, use the LinkedHashMap to store the
* order of insertion
*/
- private Map<Path, MeasurementSchema> registeredTimeseries;
+ private Map<Path, IMeasurementSchema> registeredTimeseries;
/** template name -> (measurement -> MeasurementSchema) */
- private Map<String, Map<String, MeasurementSchema>> deviceTemplates;
+ private Map<String, Map<String, IMeasurementSchema>> deviceTemplates;
public Schema() {
this.registeredTimeseries = new LinkedHashMap<>();
}
- public Schema(Map<Path, MeasurementSchema> knownSchema) {
+ public Schema(Map<Path, IMeasurementSchema> knownSchema) {
this.registeredTimeseries = knownSchema;
}
- public void registerTimeseries(Path path, MeasurementSchema descriptor) {
+ public void registerTimeseries(Path path, IMeasurementSchema descriptor) {
this.registeredTimeseries.put(path, descriptor);
}
- public void registerDeviceTemplate(String templateName, Map<String, MeasurementSchema> template) {
+ public void registerDeviceTemplate(
+ String templateName, Map<String, IMeasurementSchema> template) {
if (deviceTemplates == null) {
deviceTemplates = new HashMap<>();
}
this.deviceTemplates.put(templateName, template);
}
- public void extendTemplate(String templateName, MeasurementSchema descriptor) {
+ public void extendTemplate(String templateName, IMeasurementSchema descriptor) {
if (deviceTemplates == null) {
deviceTemplates = new HashMap<>();
}
- Map<String, MeasurementSchema> template =
+ Map<String, IMeasurementSchema> template =
this.deviceTemplates.getOrDefault(templateName, new HashMap<>());
template.put(descriptor.getMeasurementId(), descriptor);
this.deviceTemplates.put(templateName, template);
@@ -74,14 +75,14 @@
if (!deviceTemplates.containsKey(templateName)) {
return;
}
- Map<String, MeasurementSchema> template = deviceTemplates.get(templateName);
- for (Map.Entry<String, MeasurementSchema> entry : template.entrySet()) {
+ Map<String, IMeasurementSchema> template = deviceTemplates.get(templateName);
+ for (Map.Entry<String, IMeasurementSchema> entry : template.entrySet()) {
Path path = new Path(deviceId, entry.getKey());
registerTimeseries(path, entry.getValue());
}
}
- public MeasurementSchema getSeriesSchema(Path path) {
+ public IMeasurementSchema getSeriesSchema(Path path) {
return registeredTimeseries.get(path);
}
@@ -92,7 +93,7 @@
return registeredTimeseries.get(path).getType();
}
- public Map<String, Map<String, MeasurementSchema>> getDeviceTemplates() {
+ public Map<String, Map<String, IMeasurementSchema>> getDeviceTemplates() {
return deviceTemplates;
}
@@ -102,7 +103,7 @@
}
// for test
- public Map<Path, MeasurementSchema> getRegisteredTimeseriesMap() {
+ public Map<Path, IMeasurementSchema> getRegisteredTimeseriesMap() {
return registeredTimeseries;
}
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/VectorMeasurementSchema.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/VectorMeasurementSchema.java
new file mode 100644
index 0000000..e6abdc6
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/VectorMeasurementSchema.java
@@ -0,0 +1,370 @@
+/*
+ * 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.tsfile.write.schema;
+
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
+import org.apache.iotdb.tsfile.encoding.encoder.TSEncodingBuilder;
+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.utils.ReadWriteIOUtils;
+import org.apache.iotdb.tsfile.utils.StringContainer;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+public class VectorMeasurementSchema
+ implements IMeasurementSchema, Comparable<VectorMeasurementSchema>, Serializable {
+
+ public static final String VECTOR_NAME_PREFIX = "$#$";
+
+ // this is equal to the time id in this vector
+ private String vectorMeausurementId;
+ private String[] measurements;
+ private byte[] types;
+ private byte[] encodings;
+ private TSEncodingBuilder[] encodingConverters;
+ private byte compressor;
+
+ public VectorMeasurementSchema() {}
+
+ public VectorMeasurementSchema(
+ String measurementId,
+ String[] measurements,
+ TSDataType[] types,
+ TSEncoding[] encodings,
+ CompressionType compressionType) {
+ this.vectorMeausurementId = measurementId;
+ this.measurements = measurements;
+ byte[] typesInByte = new byte[types.length];
+ for (int i = 0; i < types.length; i++) {
+ typesInByte[i] = types[i].serialize();
+ }
+ this.types = typesInByte;
+
+ byte[] encodingsInByte = new byte[encodings.length];
+ for (int i = 0; i < encodings.length; i++) {
+ encodingsInByte[i] = encodings[i].serialize();
+ }
+ this.encodings = encodingsInByte;
+ this.encodingConverters = new TSEncodingBuilder[measurements.length];
+ this.compressor = compressionType.serialize();
+ }
+
+ public VectorMeasurementSchema(
+ String[] measurements, byte[] types, byte[] encodings, byte compressor) {
+ this.measurements = measurements;
+ this.types = types;
+ this.encodings = encodings;
+ this.encodingConverters = new TSEncodingBuilder[measurements.length];
+ this.compressor = compressor;
+ }
+
+ public VectorMeasurementSchema(String[] measurements, TSDataType[] types) {
+ this.measurements = measurements;
+ this.types = new byte[types.length];
+ for (int i = 0; i < types.length; i++) {
+ this.types[i] = types[i].serialize();
+ }
+
+ this.encodings = new byte[types.length];
+ for (int i = 0; i < types.length; i++) {
+ this.encodings[i] =
+ TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getValueEncoder())
+ .serialize();
+ }
+ this.encodingConverters = new TSEncodingBuilder[measurements.length];
+ this.compressor = TSFileDescriptor.getInstance().getConfig().getCompressor().serialize();
+ }
+
+ public VectorMeasurementSchema(
+ String measurementId, String[] measurements, TSDataType[] types, TSEncoding[] encodings) {
+ this(
+ measurementId,
+ measurements,
+ types,
+ encodings,
+ TSFileDescriptor.getInstance().getConfig().getCompressor());
+ }
+
+ @Override
+ public String getMeasurementId() {
+ return vectorMeausurementId;
+ }
+
+ @Override
+ public CompressionType getCompressor() {
+ return CompressionType.deserialize(compressor);
+ }
+
+ @Override
+ public TSEncoding getEncodingType() {
+ throw new UnsupportedOperationException("unsupported method for VectorMeasurementSchema");
+ }
+
+ @Override
+ public TSDataType getType() {
+ return TSDataType.VECTOR;
+ }
+
+ @Override
+ public void setType(TSDataType dataType) {
+ throw new UnsupportedOperationException("unsupported method for VectorMeasurementSchema");
+ }
+
+ @Override
+ public TSEncoding getTimeTSEncoding() {
+ return TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder());
+ }
+
+ @Override
+ public Encoder getTimeEncoder() {
+ TSEncoding timeEncoding =
+ TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder());
+ TSDataType timeType =
+ TSDataType.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeSeriesDataType());
+ return TSEncodingBuilder.getEncodingBuilder(timeEncoding).getEncoder(timeType);
+ }
+
+ @Override
+ public Encoder getValueEncoder() {
+ throw new UnsupportedOperationException("unsupported method for VectorMeasurementSchema");
+ }
+
+ @Override
+ public Map<String, String> getProps() {
+ throw new UnsupportedOperationException("unsupported method for VectorMeasurementSchema");
+ }
+
+ @Override
+ public List<String> getValueMeasurementIdList() {
+ return Arrays.asList(measurements);
+ }
+
+ @Override
+ public List<TSDataType> getValueTSDataTypeList() {
+ List<TSDataType> dataTypeList = new ArrayList<>();
+ for (byte dataType : types) {
+ dataTypeList.add(TSDataType.deserialize(dataType));
+ }
+ return dataTypeList;
+ }
+
+ @Override
+ public List<TSEncoding> getValueTSEncodingList() {
+ List<TSEncoding> encodingList = new ArrayList<>();
+ for (byte encoding : encodings) {
+ encodingList.add(TSEncoding.deserialize(encoding));
+ }
+ return encodingList;
+ }
+
+ @Override
+ public List<Encoder> getValueEncoderList() {
+ List<Encoder> encoderList = new ArrayList<>();
+ for (int i = 0; i < encodings.length; i++) {
+ TSEncoding encoding = TSEncoding.deserialize(encodings[i]);
+ // it is ok even if encodingConverter is constructed two instances for concurrent scenario
+ if (encodingConverters[i] == null) {
+ // initialize TSEncoding. e.g. set max error for PLA and SDT
+ encodingConverters[i] = TSEncodingBuilder.getEncodingBuilder(encoding);
+ encodingConverters[i].initFromProps(null);
+ }
+ encoderList.add(encodingConverters[i].getEncoder(TSDataType.deserialize(types[i])));
+ }
+ return encoderList;
+ }
+
+ @Override
+ public int getMeasurementIdColumnIndex(String measurementId) {
+ return getValueMeasurementIdList().indexOf(measurementId);
+ }
+
+ @Override
+ public int serializeTo(ByteBuffer buffer) {
+ int byteLen = 0;
+ byteLen +=
+ ReadWriteIOUtils.write(vectorMeausurementId.substring(VECTOR_NAME_PREFIX.length()), buffer);
+ byteLen += ReadWriteIOUtils.write(measurements.length, buffer);
+
+ for (String measurementId : measurements) {
+ byteLen += ReadWriteIOUtils.write(measurementId, buffer);
+ }
+ for (byte type : types) {
+ byteLen += ReadWriteIOUtils.write(type, buffer);
+ }
+ for (byte encoding : encodings) {
+ byteLen += ReadWriteIOUtils.write(encoding, buffer);
+ }
+ byteLen += ReadWriteIOUtils.write(compressor, buffer);
+
+ return byteLen;
+ }
+
+ @Override
+ public int serializeTo(OutputStream outputStream) throws IOException {
+ int byteLen = 0;
+ byteLen +=
+ ReadWriteIOUtils.write(
+ vectorMeausurementId.substring(VECTOR_NAME_PREFIX.length()), outputStream);
+ byteLen += ReadWriteIOUtils.write(measurements.length, outputStream);
+
+ for (String measurementId : measurements) {
+ byteLen += ReadWriteIOUtils.write(measurementId, outputStream);
+ }
+ for (byte type : types) {
+ byteLen += ReadWriteIOUtils.write(type, outputStream);
+ }
+ for (byte encoding : encodings) {
+ byteLen += ReadWriteIOUtils.write(encoding, outputStream);
+ }
+ byteLen += ReadWriteIOUtils.write(compressor, outputStream);
+
+ return byteLen;
+ }
+
+ @Override
+ public int partialSerializeTo(OutputStream outputStream) throws IOException {
+ ReadWriteIOUtils.write((byte) 1, outputStream);
+ return 1 + serializeTo(outputStream);
+ }
+
+ @Override
+ public int partialSerializeTo(ByteBuffer buffer) {
+ ReadWriteIOUtils.write((byte) 1, buffer);
+ return 1 + serializeTo(buffer);
+ }
+
+ public static VectorMeasurementSchema partialDeserializeFrom(ByteBuffer buffer) {
+ return deserializeFrom(buffer);
+ }
+
+ public static VectorMeasurementSchema deserializeFrom(InputStream inputStream)
+ throws IOException {
+ VectorMeasurementSchema vectorMeasurementSchema = new VectorMeasurementSchema();
+ vectorMeasurementSchema.vectorMeausurementId =
+ VECTOR_NAME_PREFIX + ReadWriteIOUtils.readString(inputStream);
+
+ int measurementSize = ReadWriteIOUtils.readInt(inputStream);
+ String[] measurements = new String[measurementSize];
+ for (int i = 0; i < measurementSize; i++) {
+ measurements[i] = ReadWriteIOUtils.readString(inputStream);
+ }
+ vectorMeasurementSchema.measurements = measurements;
+
+ byte[] types = new byte[measurementSize];
+ for (int i = 0; i < measurementSize; i++) {
+ types[i] = ReadWriteIOUtils.readByte(inputStream);
+ }
+ vectorMeasurementSchema.types = types;
+
+ byte[] encodings = new byte[measurementSize];
+ for (int i = 0; i < measurementSize; i++) {
+ encodings[i] = ReadWriteIOUtils.readByte(inputStream);
+ }
+ vectorMeasurementSchema.encodings = encodings;
+
+ vectorMeasurementSchema.compressor = ReadWriteIOUtils.readByte(inputStream);
+ return vectorMeasurementSchema;
+ }
+
+ public static VectorMeasurementSchema deserializeFrom(ByteBuffer buffer) {
+ VectorMeasurementSchema vectorMeasurementSchema = new VectorMeasurementSchema();
+ vectorMeasurementSchema.vectorMeausurementId =
+ VECTOR_NAME_PREFIX + ReadWriteIOUtils.readString(buffer);
+ int measurementSize = ReadWriteIOUtils.readInt(buffer);
+ String[] measurements = new String[measurementSize];
+ for (int i = 0; i < measurementSize; i++) {
+ measurements[i] = ReadWriteIOUtils.readString(buffer);
+ }
+ vectorMeasurementSchema.measurements = measurements;
+
+ byte[] types = new byte[measurementSize];
+ for (int i = 0; i < measurementSize; i++) {
+ types[i] = ReadWriteIOUtils.readByte(buffer);
+ }
+ vectorMeasurementSchema.types = types;
+
+ byte[] encodings = new byte[measurementSize];
+ for (int i = 0; i < measurementSize; i++) {
+ encodings[i] = ReadWriteIOUtils.readByte(buffer);
+ }
+ vectorMeasurementSchema.encodings = encodings;
+
+ vectorMeasurementSchema.compressor = ReadWriteIOUtils.readByte(buffer);
+ return vectorMeasurementSchema;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ VectorMeasurementSchema that = (VectorMeasurementSchema) o;
+ return Arrays.equals(types, that.types)
+ && Arrays.equals(encodings, that.encodings)
+ && Arrays.equals(measurements, that.measurements)
+ && Objects.equals(compressor, that.compressor);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(types, encodings, measurements, compressor);
+ }
+
+ /** compare by first measurementID. */
+ @Override
+ public int compareTo(VectorMeasurementSchema o) {
+ if (equals(o)) {
+ return 0;
+ } else {
+ return this.measurements[0].compareTo(o.measurements[0]);
+ }
+ }
+
+ @Override
+ public String toString() {
+ StringContainer sc = new StringContainer("");
+ for (int i = 0; i < measurements.length; i++) {
+ sc.addTail(
+ "[",
+ measurements[i],
+ ",",
+ TSDataType.deserialize(types[i]).toString(),
+ ",",
+ TSEncoding.deserialize(encodings[i]).toString());
+ sc.addTail("],");
+ }
+ sc.addTail(CompressionType.deserialize(compressor).toString());
+ return sc.toString();
+ }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/RestorableTsFileIOWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/RestorableTsFileIOWriter.java
index 26f8494..afedaf2 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/RestorableTsFileIOWriter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/RestorableTsFileIOWriter.java
@@ -22,12 +22,13 @@
import org.apache.iotdb.tsfile.exception.NotCompatibleTsFileException;
import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetadata;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
import org.apache.iotdb.tsfile.read.TsFileCheckStatus;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import org.apache.iotdb.tsfile.read.common.Path;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -57,7 +58,7 @@
private static final Logger logger = LoggerFactory.getLogger("FileMonitor");
private long truncatedSize = -1;
- private Map<Path, MeasurementSchema> knownSchemas = new HashMap<>();
+ private Map<Path, IMeasurementSchema> knownSchemas = new HashMap<>();
private int lastFlushedChunkGroupIndex = 0;
@@ -147,7 +148,7 @@
return truncatedSize;
}
- public Map<Path, MeasurementSchema> getKnownSchema() {
+ public Map<Path, IMeasurementSchema> getKnownSchema() {
return knownSchemas;
}
@@ -166,12 +167,12 @@
List<ChunkMetadata> chunkMetadataList = new ArrayList<>();
if (metadatasForQuery.containsKey(deviceId)
&& metadatasForQuery.get(deviceId).containsKey(measurementId)) {
- for (ChunkMetadata chunkMetaData : metadatasForQuery.get(deviceId).get(measurementId)) {
+ for (IChunkMetadata chunkMetaData : metadatasForQuery.get(deviceId).get(measurementId)) {
// filter: if a device'measurement is defined as float type, and data has been persistent.
// Then someone deletes the timeseries and recreate it with Int type. We have to ignore
// all the stale data.
if (dataType == null || dataType.equals(chunkMetaData.getDataType())) {
- chunkMetadataList.add(chunkMetaData);
+ chunkMetadataList.add((ChunkMetadata) chunkMetaData);
}
}
}
@@ -193,7 +194,7 @@
List<ChunkMetadata> rowMetaDataList = chunkGroupMetadata.getChunkMetadataList();
String device = chunkGroupMetadata.getDevice();
- for (ChunkMetadata chunkMetaData : rowMetaDataList) {
+ for (IChunkMetadata chunkMetaData : rowMetaDataList) {
String measurementId = chunkMetaData.getMeasurementUid();
if (!metadatasForQuery.containsKey(device)) {
metadatasForQuery.put(device, new HashMap<>());
@@ -201,7 +202,7 @@
if (!metadatasForQuery.get(device).containsKey(measurementId)) {
metadatasForQuery.get(device).put(measurementId, new ArrayList<>());
}
- metadatasForQuery.get(device).get(measurementId).add(chunkMetaData);
+ metadatasForQuery.get(device).get(measurementId).add((ChunkMetadata) chunkMetaData);
}
}
}
@@ -228,7 +229,7 @@
return append;
}
- public void addSchema(Path path, MeasurementSchema schema) {
+ public void addSchema(Path path, IMeasurementSchema schema) {
knownSchemas.put(path, schema);
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java
index 3df2eb9..41ed225 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java
@@ -25,6 +25,7 @@
import org.apache.iotdb.tsfile.file.header.ChunkHeader;
import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetadata;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.file.metadata.MetadataIndexConstructor;
import org.apache.iotdb.tsfile.file.metadata.MetadataIndexNode;
import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
@@ -39,7 +40,6 @@
import org.apache.iotdb.tsfile.utils.BytesUtils;
import org.apache.iotdb.tsfile.utils.PublicBAOS;
import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -120,6 +120,11 @@
startFile();
}
+ /** for test only */
+ public TsFileIOWriter(TsFileOutput output, boolean test) {
+ this.out = output;
+ }
+
/**
* Writes given bytes to output stream. This method is called when total memory size exceeds the
* chunk group size threshold.
@@ -163,35 +168,38 @@
/**
* start a {@linkplain ChunkMetadata ChunkMetaData}.
*
- * @param measurementSchema - schema of this time series
+ * @param measurementId - measurementId of this time series
* @param compressionCodecName - compression name of this time series
* @param tsDataType - data type
* @param statistics - Chunk statistics
* @param dataSize - the serialized size of all pages
+ * @param mask - 0x80 for time chunk, 0x40 for value chunk, 0x00 for common chunk
* @throws IOException if I/O error occurs
*/
public void startFlushChunk(
- MeasurementSchema measurementSchema,
+ String measurementId,
CompressionType compressionCodecName,
TSDataType tsDataType,
TSEncoding encodingType,
Statistics<?> statistics,
int dataSize,
- int numOfPages)
+ int numOfPages,
+ int mask)
throws IOException {
currentChunkMetadata =
- new ChunkMetadata(
- measurementSchema.getMeasurementId(), tsDataType, out.getPosition(), statistics);
+ new ChunkMetadata(measurementId, tsDataType, out.getPosition(), statistics);
+ currentChunkMetadata.setMask((byte) mask);
ChunkHeader header =
new ChunkHeader(
- measurementSchema.getMeasurementId(),
+ measurementId,
dataSize,
tsDataType,
compressionCodecName,
encodingType,
- numOfPages);
+ numOfPages,
+ mask);
header.serializeTo(out.wrapAsStream());
}
@@ -233,9 +241,9 @@
ReadWriteIOUtils.write(MetaMarker.SEPARATOR, out.wrapAsStream());
// group ChunkMetadata by series
- Map<Path, List<ChunkMetadata>> chunkMetadataListMap = new TreeMap<>();
+ Map<Path, List<IChunkMetadata>> chunkMetadataListMap = new TreeMap<>();
for (ChunkGroupMetadata chunkGroupMetadata : chunkGroupMetadataList) {
- for (ChunkMetadata chunkMetadata : chunkGroupMetadata.getChunkMetadataList()) {
+ for (IChunkMetadata chunkMetadata : chunkGroupMetadata.getChunkMetadataList()) {
Path series = new Path(chunkGroupMetadata.getDevice(), chunkMetadata.getMeasurementUid());
chunkMetadataListMap.computeIfAbsent(series, k -> new ArrayList<>()).add(chunkMetadata);
}
@@ -282,13 +290,13 @@
*
* @return MetadataIndexEntry list in TsFileMetadata
*/
- private MetadataIndexNode flushMetadataIndex(Map<Path, List<ChunkMetadata>> chunkMetadataListMap)
+ private MetadataIndexNode flushMetadataIndex(Map<Path, List<IChunkMetadata>> chunkMetadataListMap)
throws IOException {
// convert ChunkMetadataList to this field
deviceTimeseriesMetadataMap = new LinkedHashMap<>();
// create device -> TimeseriesMetaDataList Map
- for (Map.Entry<Path, List<ChunkMetadata>> entry : chunkMetadataListMap.entrySet()) {
+ for (Map.Entry<Path, List<IChunkMetadata>> entry : chunkMetadataListMap.entrySet()) {
Path path = entry.getKey();
String device = path.getDevice();
@@ -300,7 +308,7 @@
int chunkMetadataListLength = 0;
boolean serializeStatistic = (entry.getValue().size() > 1);
// flush chunkMetadataList one by one
- for (ChunkMetadata chunkMetadata : entry.getValue()) {
+ for (IChunkMetadata chunkMetadata : entry.getValue()) {
if (!chunkMetadata.getDataType().equals(dataType)) {
continue;
}
@@ -309,7 +317,8 @@
}
TimeseriesMetadata timeseriesMetadata =
new TimeseriesMetadata(
- serializeStatistic ? (byte) 1 : (byte) 0,
+ (byte)
+ ((serializeStatistic ? (byte) 1 : (byte) 0) | entry.getValue().get(0).getMask()),
chunkMetadataListLength,
path.getMeasurement(),
dataType,
@@ -396,7 +405,7 @@
Iterator<ChunkMetadata> chunkMetaDataIterator =
chunkGroupMetaData.getChunkMetadataList().iterator();
while (chunkMetaDataIterator.hasNext()) {
- ChunkMetadata chunkMetaData = chunkMetaDataIterator.next();
+ IChunkMetadata chunkMetaData = chunkMetaDataIterator.next();
Path path = new Path(deviceId, chunkMetaData.getMeasurementUid());
int startTimeIdx = startTimeIdxes.get(path);
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/MeasurementChunkMetadataListMapIteratorTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/MeasurementChunkMetadataListMapIteratorTest.java
index e4a1313..7c72a0d 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/MeasurementChunkMetadataListMapIteratorTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/MeasurementChunkMetadataListMapIteratorTest.java
@@ -22,6 +22,7 @@
import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.read.common.Path;
import org.apache.iotdb.tsfile.utils.FileGenerator;
@@ -125,7 +126,7 @@
List<String> devices = fileReader.getAllDevices();
- Map<String, Map<String, List<ChunkMetadata>>> expectedDeviceMeasurementChunkMetadataListMap =
+ Map<String, Map<String, List<IChunkMetadata>>> expectedDeviceMeasurementChunkMetadataListMap =
new HashMap<>();
for (String device : devices) {
for (String measurement : deviceMeasurementListMap.get(device)) {
@@ -137,10 +138,10 @@
}
for (String device : devices) {
- Map<String, List<ChunkMetadata>> expected =
+ Map<String, List<IChunkMetadata>> expected =
expectedDeviceMeasurementChunkMetadataListMap.get(device);
- Map<String, List<ChunkMetadata>> actual = new HashMap<>();
+ Map<String, List<IChunkMetadata>> actual = new HashMap<>();
Iterator<Map<String, List<ChunkMetadata>>> iterator =
fileReader.getMeasurementChunkMetadataListMapIterator(device);
while (iterator.hasNext()) {
@@ -158,11 +159,11 @@
}
private void checkCorrectness(
- Map<String, List<ChunkMetadata>> expected, Map<String, List<ChunkMetadata>> actual) {
+ Map<String, List<IChunkMetadata>> expected, Map<String, List<IChunkMetadata>> actual) {
Assert.assertEquals(expected.keySet(), actual.keySet());
for (String measurement : expected.keySet()) {
- List<ChunkMetadata> expectedChunkMetadataList = expected.get(measurement);
- List<ChunkMetadata> actualChunkMetadataList = actual.get(measurement);
+ List<IChunkMetadata> expectedChunkMetadataList = expected.get(measurement);
+ List<IChunkMetadata> actualChunkMetadataList = actual.get(measurement);
Assert.assertEquals(expectedChunkMetadataList.size(), actualChunkMetadataList.size());
final int size = expectedChunkMetadataList.size();
for (int i = 0; i < size; ++i) {
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/controller/ChunkLoaderTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/controller/ChunkLoaderTest.java
index 6d4f2ce..a510101 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/controller/ChunkLoaderTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/controller/ChunkLoaderTest.java
@@ -20,6 +20,7 @@
import org.apache.iotdb.tsfile.file.header.ChunkHeader;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import org.apache.iotdb.tsfile.read.common.Chunk;
import org.apache.iotdb.tsfile.read.common.Path;
@@ -53,12 +54,12 @@
public void test() throws IOException {
fileReader = new TsFileSequenceReader(FILE_PATH);
MetadataQuerierByFileImpl metadataQuerierByFile = new MetadataQuerierByFileImpl(fileReader);
- List<ChunkMetadata> chunkMetadataList =
+ List<IChunkMetadata> chunkMetadataList =
metadataQuerierByFile.getChunkMetaDataList(new Path("d2", "s1"));
CachedChunkLoaderImpl seriesChunkLoader = new CachedChunkLoaderImpl(fileReader);
- for (ChunkMetadata chunkMetaData : chunkMetadataList) {
- Chunk chunk = seriesChunkLoader.loadChunk(chunkMetaData);
+ for (IChunkMetadata chunkMetaData : chunkMetadataList) {
+ Chunk chunk = seriesChunkLoader.loadChunk((ChunkMetadata) chunkMetaData);
ChunkHeader chunkHeader = chunk.getHeader();
Assert.assertEquals(chunkHeader.getDataSize(), chunk.getData().remaining());
}
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/query/timegenerator/ReaderByTimestampTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/query/timegenerator/ReaderByTimestampTest.java
index 7ae6252..45d1d05 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/query/timegenerator/ReaderByTimestampTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/query/timegenerator/ReaderByTimestampTest.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.tsfile.read.query.timegenerator;
import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import org.apache.iotdb.tsfile.read.common.BatchData;
import org.apache.iotdb.tsfile.read.common.Path;
@@ -62,7 +62,7 @@
@Test
public void readByTimestamp() throws IOException {
CachedChunkLoaderImpl seriesChunkLoader = new CachedChunkLoaderImpl(fileReader);
- List<ChunkMetadata> chunkMetadataList =
+ List<IChunkMetadata> chunkMetadataList =
metadataQuerierByFile.getChunkMetaDataList(new Path("d1", "s1"));
AbstractFileSeriesReader seriesReader =
new FileSeriesReader(seriesChunkLoader, chunkMetadataList, null);
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/reader/ReaderTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/reader/ReaderTest.java
index 4d17b2b..c4abefd 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/reader/ReaderTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/reader/ReaderTest.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.tsfile.read.reader;
import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import org.apache.iotdb.tsfile.read.common.BatchData;
import org.apache.iotdb.tsfile.read.common.Path;
@@ -71,7 +71,7 @@
public void readTest() throws IOException {
int count = 0;
CachedChunkLoaderImpl seriesChunkLoader = new CachedChunkLoaderImpl(fileReader);
- List<ChunkMetadata> chunkMetadataList =
+ List<IChunkMetadata> chunkMetadataList =
metadataQuerierByFile.getChunkMetaDataList(new Path("d1", "s1"));
AbstractFileSeriesReader seriesReader =
@@ -107,7 +107,7 @@
@Test
public void readWithFilterTest() throws IOException {
CachedChunkLoaderImpl seriesChunkLoader = new CachedChunkLoaderImpl(fileReader);
- List<ChunkMetadata> chunkMetadataList =
+ List<IChunkMetadata> chunkMetadataList =
metadataQuerierByFile.getChunkMetaDataList(new Path("d1", "s1"));
Filter filter =
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/utils/BitMapTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/utils/BitMapTest.java
new file mode 100644
index 0000000..ef63b8a
--- /dev/null
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/utils/BitMapTest.java
@@ -0,0 +1,67 @@
+/*
+ * 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.tsfile.utils;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class BitMapTest {
+
+ @Test
+ public void testMarkAndUnMark() {
+ BitMap bitmap = new BitMap(100);
+ assertEquals(100, bitmap.getSize());
+ assertTrue(bitmap.isAllUnmarked());
+ assertFalse(bitmap.isAllMarked());
+ for (int i = 0; i < 100; i++) {
+ bitmap.mark(i);
+ assertTrue(bitmap.isMarked(i));
+ if (i == 50) {
+ assertFalse(bitmap.isAllMarked());
+ assertFalse(bitmap.isAllUnmarked());
+ }
+ }
+ assertTrue(bitmap.isAllMarked());
+ assertFalse(bitmap.isAllUnmarked());
+ for (int i = 0; i < 100; i++) {
+ bitmap.unmark(i);
+ assertFalse(bitmap.isMarked(i));
+ }
+ assertTrue(bitmap.isAllUnmarked());
+ assertFalse(bitmap.isAllMarked());
+ }
+
+ @Test
+ public void testInitFromBytes() {
+ BitMap bitmap1 = new BitMap(100);
+ for (int i = 0; i < 100; i++) {
+ if (i % 2 == 0) {
+ bitmap1.mark(i);
+ }
+ }
+ BitMap bitmap2 = new BitMap(bitmap1.getSize(), bitmap1.getByteArray());
+ assertEquals(100, bitmap2.getSize());
+ for (int i = 0; i < 100; i++) {
+ assertEquals(bitmap1.isMarked(i), bitmap2.isMarked(i));
+ }
+ }
+}
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/utils/RecordUtils.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/utils/RecordUtils.java
index ab5d463..ccdb6f9 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/utils/RecordUtils.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/utils/RecordUtils.java
@@ -28,7 +28,7 @@
import org.apache.iotdb.tsfile.write.record.datapoint.IntDataPoint;
import org.apache.iotdb.tsfile.write.record.datapoint.LongDataPoint;
import org.apache.iotdb.tsfile.write.record.datapoint.StringDataPoint;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.Schema;
import org.slf4j.Logger;
@@ -68,7 +68,7 @@
for (int i = 2; i < items.length - 1; i += 2) {
// get measurementId and value
measurementId = items[i].trim();
- MeasurementSchema measurementSchema =
+ IMeasurementSchema measurementSchema =
schema.getSeriesSchema(new Path(deviceId, measurementId));
if (measurementSchema == null) {
LOG.warn("measurementId:{},type not found, pass", measurementId);
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/DefaultDeviceTemplateTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/DefaultDeviceTemplateTest.java
index b8a54cb..82f7df8 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/DefaultDeviceTemplateTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/DefaultDeviceTemplateTest.java
@@ -27,6 +27,7 @@
import org.apache.iotdb.tsfile.read.expression.QueryExpression;
import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.junit.Assert;
@@ -49,11 +50,11 @@
MeasurementSchema s1 = new MeasurementSchema("s1", TSDataType.INT64, TSEncoding.PLAIN);
MeasurementSchema s2 = new MeasurementSchema("s2", TSDataType.INT64, TSEncoding.PLAIN);
- List<MeasurementSchema> schemaList = new ArrayList<>();
+ List<IMeasurementSchema> schemaList = new ArrayList<>();
schemaList.add(s1);
schemaList.add(s2);
- Map<String, MeasurementSchema> schema = new HashMap<>();
+ Map<String, IMeasurementSchema> schema = new HashMap<>();
schema.put("s1", s1);
schema.put("s2", s2);
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileIOWriterTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileIOWriterTest.java
index 570ad8e..a0eb0e9 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileIOWriterTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileIOWriterTest.java
@@ -62,12 +62,13 @@
// chunk group 1
writer.startChunkGroup(deviceId);
writer.startFlushChunk(
- measurementSchema,
+ measurementSchema.getMeasurementId(),
measurementSchema.getCompressor(),
measurementSchema.getType(),
measurementSchema.getEncodingType(),
statistics,
0,
+ 0,
0);
writer.endCurrentChunk();
writer.endChunkGroup();
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/schema/converter/SchemaBuilderTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/schema/converter/SchemaBuilderTest.java
index d21a9a6..a8c0638 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/schema/converter/SchemaBuilderTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/schema/converter/SchemaBuilderTest.java
@@ -23,6 +23,7 @@
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.Schema;
@@ -51,12 +52,12 @@
new MeasurementSchema(
"s5", TSDataType.INT32, TSEncoding.TS_2DIFF, CompressionType.UNCOMPRESSED, null));
- Collection<MeasurementSchema> timeseries = schema.getRegisteredTimeseriesMap().values();
+ Collection<IMeasurementSchema> timeseries = schema.getRegisteredTimeseriesMap().values();
String[] tsDesStrings = {
"[s4,DOUBLE,RLE,{max_point_number=3},SNAPPY]", "[s5,INT32,TS_2DIFF,,UNCOMPRESSED]"
};
int i = 0;
- for (MeasurementSchema desc : timeseries) {
+ for (IMeasurementSchema desc : timeseries) {
assertEquals(tsDesStrings[i++], desc.toString());
}
}
@@ -67,7 +68,7 @@
Map<String, String> props = new HashMap<>();
props.put(JsonFormatConstant.MAX_POINT_NUMBER, "3");
Schema schema = new Schema();
- Map<String, MeasurementSchema> template = new HashMap<>();
+ Map<String, IMeasurementSchema> template = new HashMap<>();
template.put(
"s4",
new MeasurementSchema(
@@ -79,12 +80,12 @@
schema.registerDeviceTemplate("template1", template);
schema.registerDevice("d1", "template1");
- Collection<MeasurementSchema> timeseries = schema.getRegisteredTimeseriesMap().values();
+ Collection<IMeasurementSchema> timeseries = schema.getRegisteredTimeseriesMap().values();
String[] tsDesStrings = {
"[s4,DOUBLE,RLE,{max_point_number=3},SNAPPY]", "[s5,INT32,TS_2DIFF,,UNCOMPRESSED]"
};
int i = 0;
- for (MeasurementSchema desc : timeseries) {
+ for (IMeasurementSchema desc : timeseries) {
assertEquals(tsDesStrings[i++], desc.toString());
}
}
@@ -95,7 +96,7 @@
Map<String, String> props = new HashMap<>();
props.put(JsonFormatConstant.MAX_POINT_NUMBER, "3");
Schema schema = new Schema();
- Map<String, MeasurementSchema> template = new HashMap<>();
+ Map<String, IMeasurementSchema> template = new HashMap<>();
template.put(
"s4",
new MeasurementSchema(
@@ -113,14 +114,14 @@
schema.registerDevice("d1", "template1");
- Collection<MeasurementSchema> timeseries = schema.getRegisteredTimeseriesMap().values();
+ Collection<IMeasurementSchema> timeseries = schema.getRegisteredTimeseriesMap().values();
String[] tsDesStrings = {
"[s4,DOUBLE,RLE,{max_point_number=3},SNAPPY]",
"[s5,INT32,TS_2DIFF,,UNCOMPRESSED]",
"[s6,INT64,RLE,{max_point_number=3},SNAPPY]"
};
int i = 0;
- for (MeasurementSchema desc : timeseries) {
+ for (IMeasurementSchema desc : timeseries) {
assertEquals(tsDesStrings[i++], desc.toString());
}
}
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/RestorableTsFileIOWriterTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/RestorableTsFileIOWriterTest.java
index b9ea78f..4c46c72 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/RestorableTsFileIOWriterTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/RestorableTsFileIOWriterTest.java
@@ -131,13 +131,14 @@
writer
.getIOWriter()
.startFlushChunk(
- new MeasurementSchema("s1", TSDataType.FLOAT, TSEncoding.PLAIN),
+ new MeasurementSchema("s1", TSDataType.FLOAT, TSEncoding.PLAIN).getMeasurementId(),
CompressionType.SNAPPY,
TSDataType.FLOAT,
TSEncoding.PLAIN,
new FloatStatistics(),
100,
- 10);
+ 10,
+ 0);
writer.getIOWriter().close();
RestorableTsFileIOWriter rWriter = new RestorableTsFileIOWriter(file);
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/TestTsFileOutput.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/TestTsFileOutput.java
new file mode 100644
index 0000000..f61541e
--- /dev/null
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/TestTsFileOutput.java
@@ -0,0 +1,70 @@
+/*
+ * 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.tsfile.write.writer;
+
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+public class TestTsFileOutput implements TsFileOutput {
+
+ PublicBAOS publicBAOS = new PublicBAOS();
+
+ @Override
+ public void write(byte[] b) throws IOException {
+ publicBAOS.write(b);
+ }
+
+ @Override
+ public void write(byte b) {
+ publicBAOS.write(b);
+ }
+
+ @Override
+ public void write(ByteBuffer b) {
+ publicBAOS.write(b.array(), b.position(), b.limit());
+ }
+
+ @Override
+ public long getPosition() {
+ return publicBAOS.size();
+ }
+
+ @Override
+ public void close() throws IOException {
+ publicBAOS.close();
+ }
+
+ @Override
+ public OutputStream wrapAsStream() {
+ return publicBAOS;
+ }
+
+ @Override
+ public void flush() throws IOException {
+ publicBAOS.flush();
+ }
+
+ @Override
+ public void truncate(long size) {
+ publicBAOS.truncate((int) size);
+ }
+}
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/TimeChunkWriterTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/TimeChunkWriterTest.java
new file mode 100644
index 0000000..bdca8d5
--- /dev/null
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/TimeChunkWriterTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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.tsfile.write.writer;
+
+import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
+import org.apache.iotdb.tsfile.encoding.encoder.PlainEncoder;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+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.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.iotdb.tsfile.write.chunk.TimeChunkWriter;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+
+public class TimeChunkWriterTest {
+
+ @Test
+ public void testWrite1() {
+ Encoder timeEncoder = new PlainEncoder(TSDataType.INT64, 0);
+ TimeChunkWriter chunkWriter =
+ new TimeChunkWriter("c1", CompressionType.UNCOMPRESSED, TSEncoding.PLAIN, timeEncoder);
+ for (long time = 1; time <= 10; time++) {
+ chunkWriter.write(time);
+ }
+ assertFalse(chunkWriter.checkPageSizeAndMayOpenANewPage());
+ chunkWriter.sealCurrentPage();
+ // page without statistics size: 82 + chunk header size: 8
+ assertEquals(90L, chunkWriter.getCurrentChunkSize());
+
+ try {
+ TestTsFileOutput testTsFileOutput = new TestTsFileOutput();
+ TsFileIOWriter writer = new TsFileIOWriter(testTsFileOutput, true);
+ chunkWriter.writeAllPagesOfChunkToTsFile(writer);
+ PublicBAOS publicBAOS = testTsFileOutput.publicBAOS;
+ ByteBuffer buffer = ByteBuffer.wrap(publicBAOS.getBuf(), 0, publicBAOS.size());
+ assertEquals(
+ (byte) (0x80 | MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER), ReadWriteIOUtils.readByte(buffer));
+ assertEquals("c1", ReadWriteIOUtils.readVarIntString(buffer));
+ assertEquals(82, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ assertEquals(TSDataType.VECTOR.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(CompressionType.UNCOMPRESSED.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(TSEncoding.PLAIN.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(82, buffer.remaining());
+ } catch (IOException e) {
+ e.printStackTrace();
+ fail();
+ }
+ }
+
+ @Test
+ public void testWrite2() {
+ Encoder timeEncoder = new PlainEncoder(TSDataType.INT64, 0);
+ TimeChunkWriter chunkWriter =
+ new TimeChunkWriter("c1", CompressionType.UNCOMPRESSED, TSEncoding.PLAIN, timeEncoder);
+ for (long time = 1; time <= 10; time++) {
+ chunkWriter.write(time);
+ }
+ chunkWriter.sealCurrentPage();
+ for (long time = 11; time <= 20; time++) {
+ chunkWriter.write(time);
+ }
+ chunkWriter.sealCurrentPage();
+ assertEquals(2, chunkWriter.getNumOfPages());
+ // two pages with statistics size: (82 + 17) * 2 + chunk header size: 9
+ assertEquals(207L, chunkWriter.getCurrentChunkSize());
+
+ try {
+ TestTsFileOutput testTsFileOutput = new TestTsFileOutput();
+ TsFileIOWriter writer = new TsFileIOWriter(testTsFileOutput, true);
+ chunkWriter.writeAllPagesOfChunkToTsFile(writer);
+ PublicBAOS publicBAOS = testTsFileOutput.publicBAOS;
+ ByteBuffer buffer = ByteBuffer.wrap(publicBAOS.getBuf(), 0, publicBAOS.size());
+ assertEquals((byte) (0x80 | MetaMarker.CHUNK_HEADER), ReadWriteIOUtils.readByte(buffer));
+ assertEquals("c1", ReadWriteIOUtils.readVarIntString(buffer));
+ assertEquals(198, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ assertEquals(TSDataType.VECTOR.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(CompressionType.UNCOMPRESSED.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(TSEncoding.PLAIN.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(198, buffer.remaining());
+ } catch (IOException e) {
+ e.printStackTrace();
+ fail();
+ }
+ }
+}
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/TimePageWriterTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/TimePageWriterTest.java
new file mode 100644
index 0000000..cab975c
--- /dev/null
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/TimePageWriterTest.java
@@ -0,0 +1,171 @@
+/*
+ * 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.tsfile.write.writer;
+
+import org.apache.iotdb.tsfile.compress.ICompressor;
+import org.apache.iotdb.tsfile.compress.IUnCompressor;
+import org.apache.iotdb.tsfile.encoding.decoder.PlainDecoder;
+import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
+import org.apache.iotdb.tsfile.encoding.encoder.PlainEncoder;
+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.statistics.TimeStatistics;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.iotdb.tsfile.write.page.TimePageWriter;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class TimePageWriterTest {
+
+ @Test
+ public void testWrite() {
+ Encoder timeEncoder = new PlainEncoder(TSDataType.INT64, 0);
+ ICompressor compressor = ICompressor.getCompressor(CompressionType.UNCOMPRESSED);
+ TimePageWriter pageWriter = new TimePageWriter(timeEncoder, compressor);
+ try {
+ pageWriter.write(1L);
+ assertEquals(8, pageWriter.estimateMaxMemSize());
+ ByteBuffer buffer1 = pageWriter.getUncompressedBytes();
+ ByteBuffer buffer = ByteBuffer.wrap(buffer1.array());
+ pageWriter.reset();
+ assertEquals(0, pageWriter.estimateMaxMemSize());
+ byte[] timeBytes = new byte[8];
+ buffer.get(timeBytes);
+ ByteBuffer buffer2 = ByteBuffer.wrap(timeBytes);
+ PlainDecoder decoder = new PlainDecoder();
+ assertEquals(1L, decoder.readLong(buffer2));
+ decoder.reset();
+ } catch (IOException e) {
+ fail();
+ }
+ }
+
+ @Test
+ public void testWritePageHeaderAndDataIntoBuffWithoutCompress1() {
+ Encoder timeEncoder = new PlainEncoder(TSDataType.INT64, 0);
+ ICompressor compressor = ICompressor.getCompressor(CompressionType.UNCOMPRESSED);
+ TimePageWriter pageWriter = new TimePageWriter(timeEncoder, compressor);
+ PublicBAOS publicBAOS = new PublicBAOS();
+ try {
+ pageWriter.write(1L);
+ pageWriter.write(2L);
+ pageWriter.write(3L);
+ // without page statistics
+ assertEquals(2, pageWriter.writePageHeaderAndDataIntoBuff(publicBAOS, true));
+ // total size
+ assertEquals(26, publicBAOS.size());
+ TimeStatistics statistics = pageWriter.getStatistics();
+ assertEquals(1L, statistics.getStartTime());
+ assertEquals(3L, statistics.getEndTime());
+ assertEquals(3, statistics.getCount());
+ ByteBuffer buffer = ByteBuffer.wrap(publicBAOS.getBuf(), 0, publicBAOS.size());
+ // uncompressedSize
+ assertEquals(24, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ // compressedSize
+ assertEquals(24, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ assertEquals(1L, ReadWriteIOUtils.readLong(buffer));
+ assertEquals(2L, ReadWriteIOUtils.readLong(buffer));
+ assertEquals(3L, ReadWriteIOUtils.readLong(buffer));
+ } catch (IOException e) {
+ fail();
+ }
+ }
+
+ @Test
+ public void testWritePageHeaderAndDataIntoBuffWithoutCompress2() {
+ Encoder timeEncoder = new PlainEncoder(TSDataType.INT64, 0);
+ ICompressor compressor = ICompressor.getCompressor(CompressionType.UNCOMPRESSED);
+ TimePageWriter pageWriter = new TimePageWriter(timeEncoder, compressor);
+ PublicBAOS publicBAOS = new PublicBAOS();
+ try {
+ pageWriter.write(1L);
+ pageWriter.write(2L);
+ pageWriter.write(3L);
+ // with page statistics
+ assertEquals(0, pageWriter.writePageHeaderAndDataIntoBuff(publicBAOS, false));
+ // total size
+ assertEquals(43, publicBAOS.size());
+ TimeStatistics statistics = pageWriter.getStatistics();
+ assertEquals(1L, statistics.getStartTime());
+ assertEquals(3L, statistics.getEndTime());
+ assertEquals(3, statistics.getCount());
+ ByteBuffer buffer = ByteBuffer.wrap(publicBAOS.getBuf(), 0, publicBAOS.size());
+ // uncompressedSize
+ assertEquals(24, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ // compressedSize
+ assertEquals(24, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ TimeStatistics testStatistics =
+ (TimeStatistics) TimeStatistics.deserialize(buffer, TSDataType.VECTOR);
+ assertEquals(1L, testStatistics.getStartTime());
+ assertEquals(3L, testStatistics.getEndTime());
+ assertEquals(3, testStatistics.getCount());
+ assertEquals(1L, ReadWriteIOUtils.readLong(buffer));
+ assertEquals(2L, ReadWriteIOUtils.readLong(buffer));
+ assertEquals(3L, ReadWriteIOUtils.readLong(buffer));
+ } catch (IOException e) {
+ fail();
+ }
+ }
+
+ @Test
+ public void testWritePageHeaderAndDataIntoBuffWithSnappy() {
+ Encoder timeEncoder = new PlainEncoder(TSDataType.INT64, 0);
+ ICompressor compressor = ICompressor.getCompressor(CompressionType.SNAPPY);
+ TimePageWriter pageWriter = new TimePageWriter(timeEncoder, compressor);
+ PublicBAOS publicBAOS = new PublicBAOS();
+ try {
+ pageWriter.write(1L);
+ pageWriter.write(2L);
+ pageWriter.write(3L);
+ // without page statistics
+ assertEquals(2, pageWriter.writePageHeaderAndDataIntoBuff(publicBAOS, true));
+
+ // total size
+ assertEquals(22, publicBAOS.size());
+ TimeStatistics statistics = pageWriter.getStatistics();
+ assertEquals(1L, statistics.getStartTime());
+ assertEquals(3L, statistics.getEndTime());
+ assertEquals(3, statistics.getCount());
+ ByteBuffer compressedBuffer = ByteBuffer.wrap(publicBAOS.getBuf(), 0, publicBAOS.size());
+ // uncompressedSize
+ assertEquals(24, ReadWriteForEncodingUtils.readUnsignedVarInt(compressedBuffer));
+ // compressedSize
+ assertEquals(20, ReadWriteForEncodingUtils.readUnsignedVarInt(compressedBuffer));
+ byte[] compress = new byte[20];
+ compressedBuffer.get(compress);
+ byte[] uncompress = new byte[24];
+ IUnCompressor unCompressor = IUnCompressor.getUnCompressor(CompressionType.SNAPPY);
+ unCompressor.uncompress(compress, 0, 20, uncompress, 0);
+ ByteBuffer uncompressedBuffer = ByteBuffer.wrap(uncompress);
+ assertEquals(1L, ReadWriteIOUtils.readLong(uncompressedBuffer));
+ assertEquals(2L, ReadWriteIOUtils.readLong(uncompressedBuffer));
+ assertEquals(3L, ReadWriteIOUtils.readLong(uncompressedBuffer));
+ } catch (IOException e) {
+ fail();
+ }
+ }
+}
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/ValueChunkWriterTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/ValueChunkWriterTest.java
new file mode 100644
index 0000000..3cc8272
--- /dev/null
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/ValueChunkWriterTest.java
@@ -0,0 +1,109 @@
+/*
+ * 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.tsfile.write.writer;
+
+import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
+import org.apache.iotdb.tsfile.encoding.encoder.PlainEncoder;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+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.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.iotdb.tsfile.write.chunk.ValueChunkWriter;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class ValueChunkWriterTest {
+
+ @Test
+ public void testWrite1() {
+ Encoder valueEncoder = new PlainEncoder(TSDataType.FLOAT, 0);
+ ValueChunkWriter chunkWriter =
+ new ValueChunkWriter(
+ "s1", CompressionType.UNCOMPRESSED, TSDataType.FLOAT, TSEncoding.PLAIN, valueEncoder);
+ for (int time = 1; time <= 20; time++) {
+ chunkWriter.write(time, (float) time, time % 4 == 0);
+ }
+ chunkWriter.sealCurrentPage();
+ // page without statistics size: 69 + chunk header size: 8
+ assertEquals(77L, chunkWriter.getCurrentChunkSize());
+
+ try {
+ TestTsFileOutput testTsFileOutput = new TestTsFileOutput();
+ TsFileIOWriter writer = new TsFileIOWriter(testTsFileOutput, true);
+ chunkWriter.writeAllPagesOfChunkToTsFile(writer);
+ PublicBAOS publicBAOS = testTsFileOutput.publicBAOS;
+ ByteBuffer buffer = ByteBuffer.wrap(publicBAOS.getBuf(), 0, publicBAOS.size());
+ assertEquals(0x40 | MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER, ReadWriteIOUtils.readByte(buffer));
+ assertEquals("s1", ReadWriteIOUtils.readVarIntString(buffer));
+ assertEquals(69, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ assertEquals(TSDataType.FLOAT.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(CompressionType.UNCOMPRESSED.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(TSEncoding.PLAIN.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(69, buffer.remaining());
+ } catch (IOException e) {
+ e.printStackTrace();
+ fail();
+ }
+ }
+
+ @Test
+ public void testWrite2() {
+ Encoder valueEncoder = new PlainEncoder(TSDataType.FLOAT, 0);
+ ValueChunkWriter chunkWriter =
+ new ValueChunkWriter(
+ "s1", CompressionType.UNCOMPRESSED, TSDataType.FLOAT, TSEncoding.PLAIN, valueEncoder);
+ for (int time = 1; time <= 20; time++) {
+ chunkWriter.write(time, (float) time, time % 4 == 0);
+ }
+ chunkWriter.sealCurrentPage();
+ for (int time = 20; time <= 40; time++) {
+ chunkWriter.write(time, (float) time, time % 4 == 0);
+ }
+ chunkWriter.sealCurrentPage();
+ // two pages with statistics size: (69 + 41) * 2 + chunk header size: 9
+ assertEquals(229L, chunkWriter.getCurrentChunkSize());
+
+ TestTsFileOutput testTsFileOutput = new TestTsFileOutput();
+ TsFileIOWriter writer = new TsFileIOWriter(testTsFileOutput, true);
+ try {
+ chunkWriter.writeAllPagesOfChunkToTsFile(writer);
+ PublicBAOS publicBAOS = testTsFileOutput.publicBAOS;
+ ByteBuffer buffer = ByteBuffer.wrap(publicBAOS.getBuf(), 0, publicBAOS.size());
+ assertEquals(0x40 | MetaMarker.CHUNK_HEADER, ReadWriteIOUtils.readByte(buffer));
+ assertEquals("s1", ReadWriteIOUtils.readVarIntString(buffer));
+ assertEquals(220, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ assertEquals(TSDataType.FLOAT.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(CompressionType.UNCOMPRESSED.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(TSEncoding.PLAIN.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(220, buffer.remaining());
+ } catch (IOException e) {
+ e.printStackTrace();
+ fail();
+ }
+ }
+}
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/ValuePageWriterTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/ValuePageWriterTest.java
new file mode 100644
index 0000000..a43159f
--- /dev/null
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/ValuePageWriterTest.java
@@ -0,0 +1,291 @@
+/*
+ * 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.tsfile.write.writer;
+
+import org.apache.iotdb.tsfile.compress.ICompressor;
+import org.apache.iotdb.tsfile.compress.IUnCompressor;
+import org.apache.iotdb.tsfile.encoding.decoder.PlainDecoder;
+import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
+import org.apache.iotdb.tsfile.encoding.encoder.PlainEncoder;
+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.statistics.FloatStatistics;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.iotdb.tsfile.write.page.ValuePageWriter;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class ValuePageWriterTest {
+
+ @Test
+ public void testWrite1() {
+ Encoder valueEncoder = new PlainEncoder(TSDataType.FLOAT, 0);
+ ICompressor compressor = ICompressor.getCompressor(CompressionType.UNCOMPRESSED);
+ ValuePageWriter pageWriter = new ValuePageWriter(valueEncoder, compressor, TSDataType.FLOAT);
+ try {
+ pageWriter.write(1L, 1.0f, false);
+ assertEquals(9, pageWriter.estimateMaxMemSize());
+ ByteBuffer buffer1 = pageWriter.getUncompressedBytes();
+ ByteBuffer buffer = ByteBuffer.wrap(buffer1.array());
+ pageWriter.reset(TSDataType.FLOAT);
+ assertEquals(5, pageWriter.estimateMaxMemSize());
+ assertEquals(1, ReadWriteIOUtils.readInt(buffer));
+ assertEquals(((byte) (1 << 7)), ReadWriteIOUtils.readByte(buffer));
+ PlainDecoder decoder = new PlainDecoder();
+ assertEquals(1.0f, ReadWriteIOUtils.readFloat(buffer), 0.000001f);
+ assertEquals(0, buffer.remaining());
+ decoder.reset();
+ } catch (IOException e) {
+ fail();
+ }
+ }
+
+ @Test
+ public void testWrite2() {
+ Encoder valueEncoder = new PlainEncoder(TSDataType.FLOAT, 0);
+ ICompressor compressor = ICompressor.getCompressor(CompressionType.UNCOMPRESSED);
+ ValuePageWriter pageWriter = new ValuePageWriter(valueEncoder, compressor, TSDataType.FLOAT);
+ try {
+ for (int time = 1; time <= 16; time++) {
+ pageWriter.write(time, (float) time, time % 4 == 0);
+ }
+ assertEquals(55, pageWriter.estimateMaxMemSize());
+ ByteBuffer buffer1 = pageWriter.getUncompressedBytes();
+ ByteBuffer buffer = ByteBuffer.wrap(buffer1.array());
+ pageWriter.reset(TSDataType.FLOAT);
+ assertEquals(5, pageWriter.estimateMaxMemSize());
+ assertEquals(16, ReadWriteIOUtils.readInt(buffer));
+ assertEquals(((byte) (0xEE)), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(((byte) (0xEE)), ReadWriteIOUtils.readByte(buffer));
+ PlainDecoder decoder = new PlainDecoder();
+ for (int value = 1; value <= 16; value++) {
+ if (value % 4 != 0) {
+ assertEquals((float) value, ReadWriteIOUtils.readFloat(buffer), 0.000001f);
+ }
+ }
+ assertEquals(0, buffer.remaining());
+ decoder.reset();
+ } catch (IOException e) {
+ fail();
+ }
+ }
+
+ @Test
+ public void testWrite3() {
+ Encoder valueEncoder = new PlainEncoder(TSDataType.FLOAT, 0);
+ ICompressor compressor = ICompressor.getCompressor(CompressionType.UNCOMPRESSED);
+ ValuePageWriter pageWriter = new ValuePageWriter(valueEncoder, compressor, TSDataType.FLOAT);
+ try {
+ for (int time = 1; time <= 20; time++) {
+ pageWriter.write(time, (float) time, time % 4 == 0);
+ }
+ assertEquals(67, pageWriter.estimateMaxMemSize());
+ ByteBuffer buffer1 = pageWriter.getUncompressedBytes();
+ ByteBuffer buffer = ByteBuffer.wrap(buffer1.array());
+ pageWriter.reset(TSDataType.FLOAT);
+ assertEquals(5, pageWriter.estimateMaxMemSize());
+ assertEquals(20, ReadWriteIOUtils.readInt(buffer));
+ assertEquals(((byte) (0xEE)), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(((byte) (0xEE)), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(((byte) (0xE0)), ReadWriteIOUtils.readByte(buffer));
+ PlainDecoder decoder = new PlainDecoder();
+ for (int value = 1; value <= 20; value++) {
+ if (value % 4 != 0) {
+ assertEquals((float) value, ReadWriteIOUtils.readFloat(buffer), 0.000001f);
+ }
+ }
+ assertEquals(0, buffer.remaining());
+ decoder.reset();
+ } catch (IOException e) {
+ fail();
+ }
+ }
+
+ @Test
+ public void testWritePageHeaderAndDataIntoBuffWithoutCompress1() {
+ Encoder valueEncoder = new PlainEncoder(TSDataType.FLOAT, 0);
+ ICompressor compressor = ICompressor.getCompressor(CompressionType.UNCOMPRESSED);
+ ValuePageWriter pageWriter = new ValuePageWriter(valueEncoder, compressor, TSDataType.FLOAT);
+ PublicBAOS publicBAOS = new PublicBAOS();
+ try {
+ for (int time = 1; time <= 20; time++) {
+ pageWriter.write(time, (float) time, time % 4 == 0);
+ }
+ // without page statistics
+ assertEquals(2, pageWriter.writePageHeaderAndDataIntoBuff(publicBAOS, true));
+ // total size
+ assertEquals(69, publicBAOS.size());
+ Statistics<Float> statistics = (Statistics<Float>) pageWriter.getStatistics();
+ assertEquals(1L, statistics.getStartTime());
+ assertEquals(19L, statistics.getEndTime());
+ assertEquals(15, statistics.getCount());
+ assertEquals(1.0f, statistics.getFirstValue(), 0.000001f);
+ assertEquals(19.0f, statistics.getLastValue(), 0.000001f);
+ assertEquals(1.0f, statistics.getMinValue(), 0.000001f);
+ assertEquals(19.0f, statistics.getMaxValue(), 0.000001f);
+ assertEquals(150.0f, (float) statistics.getSumDoubleValue(), 0.000001f);
+
+ ByteBuffer buffer = ByteBuffer.wrap(publicBAOS.getBuf(), 0, publicBAOS.size());
+
+ // uncompressedSize
+ assertEquals(67, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ // compressedSize
+ assertEquals(67, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+
+ // bitmap
+ assertEquals(20, ReadWriteIOUtils.readInt(buffer));
+ assertEquals(((byte) (0xEE)), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(((byte) (0xEE)), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(((byte) (0xE0)), ReadWriteIOUtils.readByte(buffer));
+
+ for (int value = 1; value <= 20; value++) {
+ if (value % 4 != 0) {
+ assertEquals((float) value, ReadWriteIOUtils.readFloat(buffer), 0.000001f);
+ }
+ }
+ assertEquals(0, buffer.remaining());
+ } catch (IOException e) {
+ fail();
+ }
+ }
+
+ @Test
+ public void testWritePageHeaderAndDataIntoBuffWithoutCompress2() {
+ Encoder valueEncoder = new PlainEncoder(TSDataType.FLOAT, 0);
+ ICompressor compressor = ICompressor.getCompressor(CompressionType.UNCOMPRESSED);
+ ValuePageWriter pageWriter = new ValuePageWriter(valueEncoder, compressor, TSDataType.FLOAT);
+ PublicBAOS publicBAOS = new PublicBAOS();
+ try {
+ for (int time = 1; time <= 20; time++) {
+ pageWriter.write(time, (float) time, time % 4 == 0);
+ }
+ // without page statistics
+ assertEquals(0, pageWriter.writePageHeaderAndDataIntoBuff(publicBAOS, false));
+ // total size
+ assertEquals(110, publicBAOS.size());
+ Statistics<Float> statistics = (Statistics<Float>) pageWriter.getStatistics();
+ assertEquals(1L, statistics.getStartTime());
+ assertEquals(19L, statistics.getEndTime());
+ assertEquals(15, statistics.getCount());
+ assertEquals(1.0f, statistics.getFirstValue(), 0.000001f);
+ assertEquals(19.0f, statistics.getLastValue(), 0.000001f);
+ assertEquals(1.0f, statistics.getMinValue(), 0.000001f);
+ assertEquals(19.0f, statistics.getMaxValue(), 0.000001f);
+ assertEquals(150.0f, (float) statistics.getSumDoubleValue(), 0.000001f);
+
+ ByteBuffer buffer = ByteBuffer.wrap(publicBAOS.getBuf(), 0, publicBAOS.size());
+ // uncompressedSize
+ assertEquals(67, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ // compressedSize
+ assertEquals(67, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+
+ // Statistics
+ FloatStatistics testStatistics =
+ (FloatStatistics) FloatStatistics.deserialize(buffer, TSDataType.FLOAT);
+ assertEquals(1L, testStatistics.getStartTime());
+ assertEquals(19L, testStatistics.getEndTime());
+ assertEquals(15, testStatistics.getCount());
+ assertEquals(1.0f, testStatistics.getFirstValue(), 0.000001f);
+ assertEquals(19.0f, testStatistics.getLastValue(), 0.000001f);
+ assertEquals(1.0f, testStatistics.getMinValue(), 0.000001f);
+ assertEquals(19.0f, testStatistics.getMaxValue(), 0.000001f);
+ assertEquals(150.0f, (float) testStatistics.getSumDoubleValue(), 0.000001f);
+
+ // bitmap
+ assertEquals(20, ReadWriteIOUtils.readInt(buffer));
+ assertEquals(((byte) (0xEE)), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(((byte) (0xEE)), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(((byte) (0xE0)), ReadWriteIOUtils.readByte(buffer));
+
+ for (int value = 1; value <= 20; value++) {
+ if (value % 4 != 0) {
+ assertEquals((float) value, ReadWriteIOUtils.readFloat(buffer), 0.000001f);
+ }
+ }
+ assertEquals(0, buffer.remaining());
+ } catch (IOException e) {
+ fail();
+ }
+ }
+
+ @Test
+ public void testWritePageHeaderAndDataIntoBuffWithSnappy() {
+ Encoder valueEncoder = new PlainEncoder(TSDataType.FLOAT, 0);
+ ICompressor compressor = ICompressor.getCompressor(CompressionType.SNAPPY);
+ ValuePageWriter pageWriter = new ValuePageWriter(valueEncoder, compressor, TSDataType.FLOAT);
+ PublicBAOS publicBAOS = new PublicBAOS();
+ try {
+ for (int time = 1; time <= 20; time++) {
+ pageWriter.write(time, (float) time, time % 4 == 0);
+ }
+ // without page statistics
+ assertEquals(2, pageWriter.writePageHeaderAndDataIntoBuff(publicBAOS, true));
+ // total size
+ assertEquals(72, publicBAOS.size());
+ Statistics<Float> statistics = (Statistics<Float>) pageWriter.getStatistics();
+ assertEquals(1L, statistics.getStartTime());
+ assertEquals(19L, statistics.getEndTime());
+ assertEquals(15, statistics.getCount());
+ assertEquals(1.0f, statistics.getFirstValue(), 0.000001f);
+ assertEquals(19.0f, statistics.getLastValue(), 0.000001f);
+ assertEquals(1.0f, statistics.getMinValue(), 0.000001f);
+ assertEquals(19.0f, statistics.getMaxValue(), 0.000001f);
+ assertEquals(150.0f, (float) statistics.getSumDoubleValue(), 0.000001f);
+
+ ByteBuffer buffer = ByteBuffer.wrap(publicBAOS.getBuf(), 0, publicBAOS.size());
+
+ // uncompressedSize
+ assertEquals(67, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ // compressedSize
+ assertEquals(70, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+
+ byte[] compress = new byte[70];
+ buffer.get(compress);
+ byte[] uncompress = new byte[67];
+ IUnCompressor unCompressor = IUnCompressor.getUnCompressor(CompressionType.SNAPPY);
+ unCompressor.uncompress(compress, 0, 70, uncompress, 0);
+ ByteBuffer uncompressedBuffer = ByteBuffer.wrap(uncompress);
+
+ // bitmap
+ assertEquals(20, ReadWriteIOUtils.readInt(uncompressedBuffer));
+ assertEquals(((byte) (0xEE)), ReadWriteIOUtils.readByte(uncompressedBuffer));
+ assertEquals(((byte) (0xEE)), ReadWriteIOUtils.readByte(uncompressedBuffer));
+ assertEquals(((byte) (0xE0)), ReadWriteIOUtils.readByte(uncompressedBuffer));
+
+ for (int value = 1; value <= 20; value++) {
+ if (value % 4 != 0) {
+ assertEquals((float) value, ReadWriteIOUtils.readFloat(uncompressedBuffer), 0.000001f);
+ }
+ }
+ assertEquals(0, uncompressedBuffer.remaining());
+ } catch (IOException e) {
+ e.printStackTrace();
+ fail();
+ }
+ }
+}
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/VectorChunkWriterImplTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/VectorChunkWriterImplTest.java
new file mode 100644
index 0000000..3ca81b1
--- /dev/null
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/VectorChunkWriterImplTest.java
@@ -0,0 +1,178 @@
+/*
+ * 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.tsfile.write.writer;
+
+import org.apache.iotdb.tsfile.file.MetaMarker;
+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.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.iotdb.tsfile.write.chunk.VectorChunkWriterImpl;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class VectorChunkWriterImplTest {
+
+ @Test
+ public void testWrite1() {
+ VectorMeasurementSchemaStub measurementSchema = new VectorMeasurementSchemaStub();
+ VectorChunkWriterImpl chunkWriter = new VectorChunkWriterImpl(measurementSchema);
+
+ for (int time = 1; time <= 20; time++) {
+ chunkWriter.write(time, (float) time, false);
+ chunkWriter.write(time, time, false);
+ chunkWriter.write(time, (double) time, false);
+ chunkWriter.write(time);
+ }
+
+ chunkWriter.sealCurrentPage();
+ // time chunk: 14 + 4 + 160; value chunk 1: 8 + 2 + 4 + 3 + 80; value chunk 2: 8 + 2 + 4 + 3 +
+ // 20; value chunk 3: 9 + 4 + 7 + 20 * 8;
+ assertEquals(492L, chunkWriter.getCurrentChunkSize());
+
+ try {
+ TestTsFileOutput testTsFileOutput = new TestTsFileOutput();
+ TsFileIOWriter writer = new TsFileIOWriter(testTsFileOutput, true);
+ chunkWriter.writeToFileWriter(writer);
+ PublicBAOS publicBAOS = testTsFileOutput.publicBAOS;
+ ByteBuffer buffer = ByteBuffer.wrap(publicBAOS.getBuf(), 0, publicBAOS.size());
+ // time chunk
+ assertEquals(
+ (byte) (0x80 | MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER), ReadWriteIOUtils.readByte(buffer));
+ assertEquals("s1.time", ReadWriteIOUtils.readVarIntString(buffer));
+ assertEquals(164, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ assertEquals(TSDataType.VECTOR.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(CompressionType.UNCOMPRESSED.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(TSEncoding.PLAIN.serialize(), ReadWriteIOUtils.readByte(buffer));
+ buffer.position(buffer.position() + 164);
+
+ // value chunk 1
+ assertEquals(0x40 | MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER, ReadWriteIOUtils.readByte(buffer));
+ assertEquals("s1", ReadWriteIOUtils.readVarIntString(buffer));
+ assertEquals(89, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ assertEquals(TSDataType.FLOAT.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(CompressionType.UNCOMPRESSED.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(TSEncoding.PLAIN.serialize(), ReadWriteIOUtils.readByte(buffer));
+ buffer.position(buffer.position() + 89);
+
+ // value chunk 2
+ assertEquals(0x40 | MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER, ReadWriteIOUtils.readByte(buffer));
+ assertEquals("s2", ReadWriteIOUtils.readVarIntString(buffer));
+ assertEquals(29, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ assertEquals(TSDataType.INT32.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(CompressionType.UNCOMPRESSED.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(TSEncoding.PLAIN.serialize(), ReadWriteIOUtils.readByte(buffer));
+ buffer.position(buffer.position() + 29);
+
+ // value chunk 2
+ assertEquals(0x40 | MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER, ReadWriteIOUtils.readByte(buffer));
+ assertEquals("s3", ReadWriteIOUtils.readVarIntString(buffer));
+ assertEquals(171, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ assertEquals(TSDataType.DOUBLE.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(CompressionType.UNCOMPRESSED.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(TSEncoding.PLAIN.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(171, buffer.remaining());
+ } catch (IOException e) {
+ e.printStackTrace();
+ fail();
+ }
+ }
+
+ @Test
+ public void testWrite2() {
+ VectorMeasurementSchemaStub measurementSchema = new VectorMeasurementSchemaStub();
+ VectorChunkWriterImpl chunkWriter = new VectorChunkWriterImpl(measurementSchema);
+
+ for (int time = 1; time <= 20; time++) {
+ chunkWriter.write(time, (float) time, false);
+ chunkWriter.write(time, time, false);
+ chunkWriter.write(time, (double) time, false);
+ chunkWriter.write(time);
+ }
+ chunkWriter.sealCurrentPage();
+ for (int time = 21; time <= 40; time++) {
+ chunkWriter.write(time, (float) time, false);
+ chunkWriter.write(time, time, false);
+ chunkWriter.write(time, (double) time, false);
+ chunkWriter.write(time);
+ }
+ chunkWriter.sealCurrentPage();
+
+ // time chunk: 14 + (4 + 17 + 160) * 2
+ // value chunk 1: 9 + (2 + 41 + 4 + 3 + 80) * 2
+ // value chunk 2: 9 + (2 + 41 + 4 + 3 + 20) * 2
+ // value chunk 3: 9 + (4 + 57 + 4 + 3 + 160) * 2
+ assertEquals(1259L, chunkWriter.getCurrentChunkSize());
+
+ try {
+ TestTsFileOutput testTsFileOutput = new TestTsFileOutput();
+ TsFileIOWriter writer = new TsFileIOWriter(testTsFileOutput, true);
+ chunkWriter.writeToFileWriter(writer);
+ PublicBAOS publicBAOS = testTsFileOutput.publicBAOS;
+ ByteBuffer buffer = ByteBuffer.wrap(publicBAOS.getBuf(), 0, publicBAOS.size());
+ // time chunk
+ assertEquals((byte) (0x80 | MetaMarker.CHUNK_HEADER), ReadWriteIOUtils.readByte(buffer));
+ assertEquals("s1.time", ReadWriteIOUtils.readVarIntString(buffer));
+ assertEquals(362, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ assertEquals(TSDataType.VECTOR.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(CompressionType.UNCOMPRESSED.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(TSEncoding.PLAIN.serialize(), ReadWriteIOUtils.readByte(buffer));
+ buffer.position(buffer.position() + 362);
+
+ // value chunk 1
+ assertEquals(0x40 | MetaMarker.CHUNK_HEADER, ReadWriteIOUtils.readByte(buffer));
+ assertEquals("s1", ReadWriteIOUtils.readVarIntString(buffer));
+ assertEquals(260, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ assertEquals(TSDataType.FLOAT.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(CompressionType.UNCOMPRESSED.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(TSEncoding.PLAIN.serialize(), ReadWriteIOUtils.readByte(buffer));
+ buffer.position(buffer.position() + 260);
+
+ // value chunk 2
+ assertEquals(0x40 | MetaMarker.CHUNK_HEADER, ReadWriteIOUtils.readByte(buffer));
+ assertEquals("s2", ReadWriteIOUtils.readVarIntString(buffer));
+ assertEquals(140, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ assertEquals(TSDataType.INT32.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(CompressionType.UNCOMPRESSED.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(TSEncoding.PLAIN.serialize(), ReadWriteIOUtils.readByte(buffer));
+ buffer.position(buffer.position() + 140);
+
+ // value chunk 2
+ assertEquals(0x40 | MetaMarker.CHUNK_HEADER, ReadWriteIOUtils.readByte(buffer));
+ assertEquals("s3", ReadWriteIOUtils.readVarIntString(buffer));
+ assertEquals(456, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
+ assertEquals(TSDataType.DOUBLE.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(CompressionType.UNCOMPRESSED.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(TSEncoding.PLAIN.serialize(), ReadWriteIOUtils.readByte(buffer));
+ assertEquals(456, buffer.remaining());
+
+ } catch (IOException e) {
+ e.printStackTrace();
+ fail();
+ }
+ }
+}
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/VectorMeasurementSchemaStub.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/VectorMeasurementSchemaStub.java
new file mode 100644
index 0000000..d4fbef2
--- /dev/null
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/VectorMeasurementSchemaStub.java
@@ -0,0 +1,128 @@
+/*
+ * 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.tsfile.write.writer;
+
+import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
+import org.apache.iotdb.tsfile.encoding.encoder.PlainEncoder;
+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.write.schema.IMeasurementSchema;
+
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+public class VectorMeasurementSchemaStub implements IMeasurementSchema {
+
+ @Override
+ public String getMeasurementId() {
+ return "s1.time";
+ }
+
+ @Override
+ public CompressionType getCompressor() {
+ return CompressionType.UNCOMPRESSED;
+ }
+
+ @Override
+ public TSEncoding getEncodingType() {
+ return null;
+ }
+
+ @Override
+ public TSDataType getType() {
+ return TSDataType.VECTOR;
+ }
+
+ @Override
+ public void setType(TSDataType dataType) {
+ throw new UnsupportedOperationException("unsupported method for VectorMeasurementSchema");
+ }
+
+ @Override
+ public TSEncoding getTimeTSEncoding() {
+ return TSEncoding.PLAIN;
+ }
+
+ @Override
+ public Encoder getTimeEncoder() {
+ return new PlainEncoder(TSDataType.INT64, 0);
+ }
+
+ @Override
+ public Encoder getValueEncoder() {
+ return null;
+ }
+
+ @Override
+ public Map<String, String> getProps() {
+ return null;
+ }
+
+ @Override
+ public List<String> getValueMeasurementIdList() {
+ return Arrays.asList("s1", "s2", "s3");
+ }
+
+ @Override
+ public List<TSDataType> getValueTSDataTypeList() {
+ return Arrays.asList(TSDataType.FLOAT, TSDataType.INT32, TSDataType.DOUBLE);
+ }
+
+ @Override
+ public List<TSEncoding> getValueTSEncodingList() {
+ return Arrays.asList(TSEncoding.PLAIN, TSEncoding.PLAIN, TSEncoding.PLAIN);
+ }
+
+ @Override
+ public List<Encoder> getValueEncoderList() {
+ return Arrays.asList(
+ new PlainEncoder(TSDataType.FLOAT, 0),
+ new PlainEncoder(TSDataType.INT32, 0),
+ new PlainEncoder(TSDataType.DOUBLE, 0));
+ }
+
+ @Override
+ public int serializeTo(ByteBuffer buffer) {
+ return 0;
+ }
+
+ @Override
+ public int serializeTo(OutputStream outputStream) {
+ return 0;
+ }
+
+ @Override
+ public int partialSerializeTo(OutputStream outputStream) {
+ return 0;
+ }
+
+ @Override
+ public int partialSerializeTo(ByteBuffer buffer) {
+ return 0;
+ }
+
+ @Override
+ public int getMeasurementIdColumnIndex(String measurementId) {
+ return 0;
+ }
+}