You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2020/02/27 08:53:41 UTC

[incubator-iotdb] branch fix_timegenerator created (now 95216c0)

This is an automated email from the ASF dual-hosted git repository.

qiaojialin pushed a change to branch fix_timegenerator
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git.


      at 95216c0  fix timegenerator

This branch includes the following new commits:

     new 12b7404  Merge remote-tracking branch 'origin/master'
     new 5370b30  fix set storage group grammar
     new 95216c0  fix timegenerator

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[incubator-iotdb] 02/03: fix set storage group grammar

Posted by qi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

qiaojialin pushed a commit to branch fix_timegenerator
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git

commit 5370b30261b54b9f3bc5305478581912285462b8
Author: qiaojialin <64...@qq.com>
AuthorDate: Thu Feb 27 15:47:49 2020 +0800

    fix set storage group grammar
---
 server/src/assembly/resources/conf/iotdb-engine.properties |  2 +-
 .../main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4 | 14 +++++++-------
 .../org/apache/iotdb/db/qp/strategy/LogicalGenerator.java  | 10 +++++-----
 3 files changed, 13 insertions(+), 13 deletions(-)

diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties
index f9906c7..eb7897c 100644
--- a/server/src/assembly/resources/conf/iotdb-engine.properties
+++ b/server/src/assembly/resources/conf/iotdb-engine.properties
@@ -430,7 +430,7 @@ compressor=SNAPPY
 # Time series data will divide into groups by this time range
 # Unit is second
 # default value is one week, which is 604800 seconds
-partition_interval = 604800
+partition_interval=604800
 
 # the num of memtables in each storage group
 # This config decides how many time partitions in a storage group can be inserted concurrently
diff --git a/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4 b/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
index 81b69c4..3665f1e 100644
--- a/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
+++ b/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
@@ -24,17 +24,17 @@ singleStatement
     ;
 
 statement
-    : CREATE TIMESERIES timeseriesPath WITH attributeClauses #createTimeseries
+    : CREATE TIMESERIES fullPath WITH attributeClauses #createTimeseries
     | DELETE TIMESERIES prefixPath (COMMA prefixPath)* #deleteTimeseries
-    | INSERT INTO timeseriesPath insertColumnSpec VALUES insertValuesSpec #insertStatement
+    | INSERT INTO fullPath insertColumnSpec VALUES insertValuesSpec #insertStatement
     | UPDATE prefixPath setClause whereClause? #updateStatement
     | DELETE FROM prefixPath (COMMA prefixPath)* (whereClause)? #deleteStatement
-    | SET STORAGE GROUP TO prefixPath #setStorageGroup
+    | SET STORAGE GROUP TO fullPath #setStorageGroup
     | DELETE STORAGE GROUP prefixPath (COMMA prefixPath)* #deleteStorageGroup
     | SHOW METADATA #showMetadata // not support yet
     | DESCRIBE prefixPath #describePath // not support yet
-    | CREATE INDEX ON timeseriesPath USING function=ID indexWithClause? whereClause? #createIndex //not support yet
-    | DROP INDEX function=ID ON timeseriesPath #dropIndex //not support yet
+    | CREATE INDEX ON fullPath USING function=ID indexWithClause? whereClause? #createIndex //not support yet
+    | DROP INDEX function=ID ON fullPath #dropIndex //not support yet
     | MERGE #merge //not support yet
     | CREATE USER userName=ID password=STRING_LITERAL #createUser
     | ALTER USER userName=(ROOT|ID) SET PASSWORD password=STRING_LITERAL #alterUser
@@ -77,7 +77,7 @@ statement
     | MOVE FILE FILE #moveFile
     | SELECT INDEX func=ID //not support yet
     LR_BRACKET
-    p1=timeseriesPath COMMA p2=timeseriesPath COMMA n1=timeValue COMMA n2=timeValue COMMA
+    p1=fullPath COMMA p2=fullPath COMMA n1=timeValue COMMA n2=timeValue COMMA
     epsilon=constant (COMMA alpha=constant COMMA beta=constant)?
     RR_BRACKET
     fromClause
@@ -271,7 +271,7 @@ propertyLabelPair
     : propertyName=ID DOT labelName=ID
     ;
 
-timeseriesPath
+fullPath
     : ROOT (DOT nodeNameWithoutStar)*
     ;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
index 9c5d837..318d1b6 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
@@ -80,6 +80,7 @@ import org.apache.iotdb.db.qp.strategy.SqlBaseParser.DropRoleContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.DropUserContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.FillClauseContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.FromClauseContext;
+import org.apache.iotdb.db.qp.strategy.SqlBaseParser.FullPathContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.FunctionCallContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.FunctionElementContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.GrantRoleContext;
@@ -135,7 +136,6 @@ import org.apache.iotdb.db.qp.strategy.SqlBaseParser.SlimitClauseContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.SoffsetClauseContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.SuffixPathContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.TimeIntervalContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.TimeseriesPathContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.TypeClauseContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.UnsetTTLStatementContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.UpdateStatementContext;
@@ -298,7 +298,7 @@ public class LogicalGenerator extends SqlBaseBaseListener {
     super.enterCreateTimeseries(ctx);
     createTimeSeriesOperator = new CreateTimeSeriesOperator(SQLConstant.TOK_METADATA_CREATE);
     operatorType = SQLConstant.TOK_METADATA_CREATE;
-    createTimeSeriesOperator.setPath(parseTimeseriesPath(ctx.timeseriesPath()));
+    createTimeSeriesOperator.setPath(parseFullPath(ctx.fullPath()));
   }
 
   @Override
@@ -624,7 +624,7 @@ public class LogicalGenerator extends SqlBaseBaseListener {
     super.enterSetStorageGroup(ctx);
     SetStorageGroupOperator setStorageGroupOperator = new SetStorageGroupOperator(
         SQLConstant.TOK_METADATA_SET_FILE_LEVEL);
-    Path path = parsePrefixPath(ctx.prefixPath());
+    Path path = parseFullPath(ctx.fullPath());
     setStorageGroupOperator.setPath(path);
     initializedOperator = setStorageGroupOperator;
     operatorType = SQLConstant.TOK_METADATA_SET_FILE_LEVEL;
@@ -870,7 +870,7 @@ public class LogicalGenerator extends SqlBaseBaseListener {
     initializedOperator = insertOp;
   }
 
-  private Path parseTimeseriesPath(TimeseriesPathContext ctx) {
+  private Path parseFullPath(FullPathContext ctx) {
     List<NodeNameWithoutStarContext> nodeNamesWithoutStar = ctx.nodeNameWithoutStar();
     List<String> path = new ArrayList<>();
     if (ctx.ROOT() != null) {
@@ -916,7 +916,7 @@ public class LogicalGenerator extends SqlBaseBaseListener {
     insertOp = new InsertOperator(SQLConstant.TOK_INSERT);
     selectOp = new SelectOperator(SQLConstant.TOK_SELECT);
     operatorType = SQLConstant.TOK_INSERT;
-    selectOp.addSelectPath(parseTimeseriesPath(ctx.timeseriesPath()));
+    selectOp.addSelectPath(parseFullPath(ctx.fullPath()));
     insertOp.setSelectOperator(selectOp);
   }
 


[incubator-iotdb] 03/03: fix timegenerator

Posted by qi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

qiaojialin pushed a commit to branch fix_timegenerator
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git

commit 95216c03499b6d405a8ee6ed79b4e497fd502577
Author: qiaojialin <64...@qq.com>
AuthorDate: Thu Feb 27 16:53:23 2020 +0800

    fix timegenerator
---
 .../read/query/timegenerator/node/AndNode.java     | 12 ++++
 .../read/query/timegenerator/node/LeafNode.java    |  9 ++-
 .../tsfile/read/query/timegenerator/NodeTest.java  | 66 +++++-----------------
 .../iotdb/tsfile/read/reader/FakedBatchReader.java | 55 ++++++++++++++++++
 4 files changed, 88 insertions(+), 54 deletions(-)

diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/node/AndNode.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/node/AndNode.java
index fa679c1..8088b54 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/node/AndNode.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/node/AndNode.java
@@ -58,6 +58,10 @@ public class AndNode implements Node {
     fillLeftCache();
     fillRightCache();
 
+    if (!hasLeftValue() || !hasRightValue()) {
+      return false;
+    }
+
     while (leftTimeColumn.hasCurrent() && rightTimeColumn.hasCurrent()) {
       long leftValue = leftTimeColumn.currentTime();
       long rightValue = rightTimeColumn.currentTime();
@@ -94,6 +98,14 @@ public class AndNode implements Node {
     }
   }
 
+  private boolean hasLeftValue() {
+    return leftTimeColumn != null && leftTimeColumn.hasCurrent();
+  }
+
+  private boolean hasRightValue() {
+    return rightTimeColumn != null && rightTimeColumn.hasCurrent();
+  }
+
   //no more data in cache and has more data in child
   private boolean couldFillCache(TimeColumn timeSeries, Node child) throws IOException {
     return (timeSeries == null || !timeSeries.hasCurrent()) && child.hasNextTimeColumn();
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/node/LeafNode.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/node/LeafNode.java
index 3f255af..b389fa4 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/node/LeafNode.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/node/LeafNode.java
@@ -40,10 +40,13 @@ public class LeafNode implements Node {
     if (hasCached) {
       return true;
     }
-    if (reader.hasNextBatch()) {
-      hasCached = true;
+    while (reader.hasNextBatch()) {
       cacheData = reader.nextBatch();
-      cachedTimeSeries = cacheData.getTimeColumn();
+      if (cacheData.hasCurrent()) {
+        hasCached = true;
+        cachedTimeSeries = cacheData.getTimeColumn();
+        break;
+      }
     }
     return hasCached;
   }
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/query/timegenerator/NodeTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/query/timegenerator/NodeTest.java
index e4ad037..6037993 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/query/timegenerator/NodeTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/query/timegenerator/NodeTest.java
@@ -18,15 +18,14 @@
  */
 package org.apache.iotdb.tsfile.read.query.timegenerator;
 
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.common.TimeColumn;
 import org.apache.iotdb.tsfile.read.query.timegenerator.node.AndNode;
 import org.apache.iotdb.tsfile.read.query.timegenerator.node.LeafNode;
 import org.apache.iotdb.tsfile.read.query.timegenerator.node.Node;
 import org.apache.iotdb.tsfile.read.query.timegenerator.node.NodeType;
 import org.apache.iotdb.tsfile.read.query.timegenerator.node.OrNode;
+import org.apache.iotdb.tsfile.read.reader.FakedBatchReader;
+import org.apache.iotdb.tsfile.read.reader.IBatchReader;
 import org.apache.iotdb.tsfile.read.reader.series.AbstractFileSeriesReader;
 import org.junit.Assert;
 import org.junit.Test;
@@ -44,13 +43,17 @@ public class NodeTest {
 
   @Test
   public void testLeafNode() throws IOException {
-    int index = 0;
     long[] timestamps = new long[]{1, 2, 3, 4, 5, 6, 7};
-    AbstractFileSeriesReader seriesReader = new FakedFileSeriesReader(timestamps);
-    Node leafNode = new LeafNode(seriesReader);
-    while (leafNode.hasNextTimeColumn()) {
-      Assert.assertEquals(timestamps[index++], leafNode.nextTimeColumn().currentTime());
+    IBatchReader batchReader = new FakedBatchReader(timestamps);
+    Node leafNode = new LeafNode(batchReader);
+
+    Assert.assertTrue(leafNode.hasNextTimeColumn());
+    TimeColumn timeColumn = leafNode.nextTimeColumn();
+    for (long timestamp : timestamps) {
+      Assert.assertEquals(timestamp, timeColumn.currentTime());
+      timeColumn.next();
     }
+    Assert.assertFalse(leafNode.hasNextTimeColumn());
   }
 
   @Test
@@ -69,8 +72,8 @@ public class NodeTest {
 
   private void testOr(long[] ret, long[] left, long[] right) throws IOException {
     int index = 0;
-    Node orNode = new OrNode(new LeafNode(new FakedFileSeriesReader(left)),
-        new LeafNode(new FakedFileSeriesReader(right)));
+    Node orNode = new OrNode(new LeafNode(new FakedBatchReader(left)),
+        new LeafNode(new FakedBatchReader(right)));
     while (orNode.hasNextTimeColumn()) {
       TimeColumn timeSeries = orNode.nextTimeColumn();
       while (timeSeries.hasCurrent()) {
@@ -93,8 +96,8 @@ public class NodeTest {
 
   private void testAnd(long[] ret, long[] left, long[] right) throws IOException {
     int index = 0;
-    Node andNode = new AndNode(new LeafNode(new FakedFileSeriesReader(left)),
-        new LeafNode(new FakedFileSeriesReader(right)));
+    Node andNode = new AndNode(new LeafNode(new FakedBatchReader(left)),
+        new LeafNode(new FakedBatchReader(right)));
     while (andNode.hasNextTimeColumn()) {
       TimeColumn timeSeries = andNode.nextTimeColumn();
       while (timeSeries.hasCurrent()) {
@@ -106,44 +109,5 @@ public class NodeTest {
     Assert.assertEquals(ret.length, index);
   }
 
-  private static class FakedFileSeriesReader extends AbstractFileSeriesReader {
-
-    BatchData data;
-    boolean hasCachedData;
-
-    public FakedFileSeriesReader(long[] timestamps) {
-      super(null, null, null);
-      data = new BatchData(TSDataType.INT32);
-      for (long time : timestamps) {
-        data.putInt(time, 1);
-      }
-      hasCachedData = true;
-    }
-
-    @Override
-    public boolean hasNextBatch() {
-      return hasCachedData;
-    }
-
-    @Override
-    public BatchData nextBatch() {
-      hasCachedData = false;
-      return data;
-    }
 
-    @Override
-    protected void initChunkReader(ChunkMetaData chunkMetaData) throws IOException {
-
-    }
-
-    @Override
-    protected boolean chunkSatisfied(ChunkMetaData chunkMetaData) {
-      return false;
-    }
-
-    @Override
-    public void close() {
-
-    }
-  }
 }
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/reader/FakedBatchReader.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/reader/FakedBatchReader.java
new file mode 100644
index 0000000..f9683d1
--- /dev/null
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/reader/FakedBatchReader.java
@@ -0,0 +1,55 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class FakedBatchReader implements IBatchReader {
+
+  private BatchData data;
+  private boolean hasCached = false;
+
+  public FakedBatchReader(long[] timestamps) {
+    data = new BatchData(TSDataType.INT32);
+    for (long time : timestamps) {
+      data.putInt(time, 1);
+      hasCached = true;
+    }
+  }
+
+  @Override
+  public boolean hasNextBatch() {
+    return hasCached;
+  }
+
+  @Override
+  public BatchData nextBatch() throws IOException {
+    if (data == null || !data.hasCurrent()) {
+      throw new IOException("no next batch");
+    }
+    hasCached = false;
+    return data;
+  }
+
+  @Override
+  public void close() {
+  }
+}


[incubator-iotdb] 01/03: Merge remote-tracking branch 'origin/master'

Posted by qi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

qiaojialin pushed a commit to branch fix_timegenerator
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git

commit 12b7404cc47efcbde7f94ea1a99baddedb9bd558
Merge: 5d44648 bcd6c22
Author: qiaojialin <64...@qq.com>
AuthorDate: Thu Feb 27 15:36:32 2020 +0800

    Merge remote-tracking branch 'origin/master'

 .../2-DML (Data Manipulation Language).md          | 23 ++++++++-
 .../5-Operation Manual/4-SQL Reference.md          | 19 +++----
 .../2-DML (Data Manipulation Language).md          | 19 ++++++-
 .../5-Operation Manual/4-SQL Reference.md          |  6 ++-
 .../main/java/org/apache/iotdb/JDBCExample.java    |  2 +-
 .../org/apache/iotdb/jdbc/IoTDBResultMetadata.java |  7 +--
 .../apache/iotdb/jdbc/IoTDBResultMetadataTest.java | 30 ++++++-----
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |  6 ++-
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |  6 ---
 .../db/query/aggregation/impl/AvgAggrResult.java   |  6 +++
 .../iotdb/db/integration/IoTDBSimpleQueryTest.java | 59 ++++++++++++++++++++++
 .../db/query/aggregation/AggregateResultTest.java  |  2 +
 .../tsfile/read/query/dataset/EmptyDataSet.java    | 39 ++++++++++++++
 13 files changed, 184 insertions(+), 40 deletions(-)