You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2020/01/14 08:42:08 UTC

[GitHub] [incubator-iotdb] JackieTien97 opened a new pull request #738: [IOTDB-396] New query clause: disable align

JackieTien97 opened a new pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366227391
 
 

 ##########
 File path: jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBNonAlignQueryResultSet.java
 ##########
 @@ -0,0 +1,1289 @@
+/*
+ * 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.jdbc;
+
+import org.apache.iotdb.rpc.IoTDBRPCException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.service.rpc.thrift.*;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.thrift.TException;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.sql.Date;
+import java.sql.*;
+import java.util.*;
+
+public class IoTDBNonAlignQueryResultSet implements ResultSet {
+
+  private static final String TIMESTAMP_STR = "Time";
+  private static final int TIMESTAMP_STR_LENGTH = 4;
+  private static final int START_INDEX = 2;
+  private static final String VALUE_IS_NULL = "The value got by %s (column name) is NULL.";
+  private static final String EMPTY_STR = "";
+  private Statement statement = null;
+  private String sql;
+  private SQLWarning warningChain = null;
+  private boolean isClosed = false;
+  private TSIService.Iface client = null;
+  private List<String> columnInfoList; // no deduplication
+  private List<String> columnTypeList; // no deduplication
+  private Map<String, Integer> columnInfoMap; // used because the server returns deduplicated columns
 
 Review comment:
   It would be more clear to call it "columnBufferPositionMap" or "columnOrdinalMap" or something else.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366289729
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/NonAlignEngineDataSet.java
 ##########
 @@ -0,0 +1,354 @@
+/*
+ * 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.dataset;
+
+import org.apache.iotdb.db.query.pool.QueryTaskPoolManager;
+import org.apache.iotdb.db.query.reader.ManagedSeriesReader;
+import org.apache.iotdb.db.tools.watermark.WatermarkEncoder;
+import org.apache.iotdb.service.rpc.thrift.TSQueryNonAlignDataSet;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicIntegerArray;
+
+public class NonAlignEngineDataSet extends QueryDataSet {
+
+  private static class ReadTask implements Runnable {
+
+    private final ManagedSeriesReader reader;
+    private BlockingQueue<Pair<ByteBuffer, ByteBuffer>> blockingQueue;
+    private WatermarkEncoder encoder;
+    NonAlignEngineDataSet dataSet;
+    private int index;
+
+
+    public ReadTask(ManagedSeriesReader reader,
+                    BlockingQueue<Pair<ByteBuffer, ByteBuffer>> blockingQueue,
+                    WatermarkEncoder encoder, NonAlignEngineDataSet dataSet, int index) {
+      this.reader = reader;
+      this.blockingQueue = blockingQueue;
+      this.encoder = encoder;
+      this.dataSet = dataSet;
+      this.index = index;
+    }
+
+    @Override
+    public void run() {
+      PublicBAOS timeBAOS = new PublicBAOS();
+      PublicBAOS valueBAOS = new PublicBAOS();
+      try {
+        synchronized (reader) {
+          // if the task is submitted, there must be free space in the queue
+          // so here we don't need to check whether the queue has free space
+          // the reader has next batch
+          if ((dataSet.cachedBatchData[index] != null && dataSet.cachedBatchData[index].hasCurrent())
+                  || reader.hasNextBatch()) {
+            BatchData batchData;
+            if (dataSet.cachedBatchData[index] != null && dataSet.cachedBatchData[index].hasCurrent())
+              batchData = dataSet.cachedBatchData[index];
+            else
+              batchData = reader.nextBatch();
+
+            int rowCount = 0;
+            while (rowCount < dataSet.fetchSize) {
+
+              if ((dataSet.limit > 0 && dataSet.alreadyReturnedRowNumArray.get(index) >= dataSet.limit)) {
+                break;
+              }
+
+              if (batchData != null && batchData.hasCurrent()) {
+                if (dataSet.offsetArray.get(index) == 0) {
+                  long time = batchData.currentTime();
+                  ReadWriteIOUtils.write(time, timeBAOS);
+                  TSDataType type = batchData.getDataType();
+                  switch (type) {
+                    case INT32:
+                      int intValue = batchData.getInt();
+                      if (encoder != null && encoder.needEncode(time)) {
+                        intValue = encoder.encodeInt(intValue, time);
+                      }
+                      ReadWriteIOUtils.write(intValue, valueBAOS);
+                      break;
+                    case INT64:
+                      long longValue = batchData.getLong();
+                      if (encoder != null && encoder.needEncode(time)) {
+                        longValue = encoder.encodeLong(longValue, time);
+                      }
+                      ReadWriteIOUtils.write(longValue, valueBAOS);
+                      break;
+                    case FLOAT:
+                      float floatValue = batchData.getFloat();
+                      if (encoder != null && encoder.needEncode(time)) {
+                        floatValue = encoder.encodeFloat(floatValue, time);
+                      }
+                      ReadWriteIOUtils.write(floatValue, valueBAOS);
+                      break;
+                    case DOUBLE:
+                      double doubleValue = batchData.getDouble();
+                      if (encoder != null && encoder.needEncode(time)) {
+                        doubleValue = encoder.encodeDouble(doubleValue, time);
+                      }
+                      ReadWriteIOUtils.write(doubleValue, valueBAOS);
+                      break;
+                    case BOOLEAN:
+                      ReadWriteIOUtils.write(batchData.getBoolean(),
+                              valueBAOS);
+                      break;
+                    case TEXT:
+                      ReadWriteIOUtils
+                              .write(batchData.getBinary(),
+                                      valueBAOS);
+                      break;
+                    default:
+                      throw new UnSupportedDataTypeException(
+                              String.format("Data type %s is not supported.", type));
+                  }
+                }
+                batchData.next();
+              }
+              else {
+                if (reader.hasNextBatch()) {
+                  batchData = reader.nextBatch();
+                  dataSet.cachedBatchData[index] = batchData;
+                  continue;
+                }
+                else
+                  break;
+              }
+              if (dataSet.offsetArray.get(index) == 0) {
+                rowCount++;
+                if (dataSet.limit > 0) {
+                  dataSet.alreadyReturnedRowNumArray.incrementAndGet(index);
+                }
+              } else {
+                dataSet.offsetArray.decrementAndGet(index);
+              }
+            }
+            if (rowCount == 0) {
+              blockingQueue.put(new Pair(null, null));
+              // set the hasRemaining field in reader to false
+              // tell the Consumer not to submit another task for this reader any more
+              reader.setHasRemaining(false);
+              // remove itself from the QueryTaskPoolManager
+              reader.setManagedByQueryManager(false);
+              return;
+            }
+
+            ByteBuffer timeBuffer = ByteBuffer.allocate(timeBAOS.size());
+            timeBuffer.put(timeBAOS.getBuf(), 0, timeBAOS.size());
+            timeBuffer.flip();
+            ByteBuffer valueBuffer = ByteBuffer.allocate(valueBAOS.size());
+            valueBuffer.put(valueBAOS.getBuf(), 0, valueBAOS.size());
+            valueBuffer.flip();
+
+            Pair<ByteBuffer, ByteBuffer> timeValueBAOSPair = new Pair(timeBuffer, valueBuffer);
+
+            blockingQueue.put(timeValueBAOSPair);
+            // if the queue also has free space, just submit another itself
+            if (blockingQueue.remainingCapacity() > 0) {
+              pool.submit(this);
+            }
+            // the queue has no more space
+            // remove itself from the QueryTaskPoolManager
+            else {
+              reader.setManagedByQueryManager(false);
+            }
+            return;
+          }
+          blockingQueue.put(new Pair(null, null));
+          // set the hasRemaining field in reader to false
+          // tell the Consumer not to submit another task for this reader any more
+          reader.setHasRemaining(false);
+          // remove itself from the QueryTaskPoolManager
+          reader.setManagedByQueryManager(false);
+        }
+      } catch (InterruptedException e) {
+        LOGGER.error("Interrupted while putting into the blocking queue: ", e);
+      } catch (IOException e) {
+        LOGGER.error("Something gets wrong while reading from the series reader: ", e);
+      } catch (Exception e) {
+        LOGGER.error("Something gets wrong: ", e);
+      }
+
+    }
+
+  }
+
+
+  private List<ManagedSeriesReader> seriesReaderWithoutValueFilterList;
+
+  // Blocking queue list for each time value buffer pair
+  private BlockingQueue<Pair<ByteBuffer, ByteBuffer>>[] blockingQueueArray;
+
+  private boolean initialized = false;
+
+  private AtomicIntegerArray offsetArray;
+
+  private int limit;
+
+  private AtomicIntegerArray alreadyReturnedRowNumArray;
 
 Review comment:
   Is it necessary to use atomic objects? Will there be concurrent modifications on the same index? I thought all read tasks are working on different indexes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366219463
 
 

 ##########
 File path: client/src/main/java/org/apache/iotdb/client/AbstractClient.java
 ##########
 @@ -297,13 +324,24 @@ private static void printColumnData(ResultSetMetaData resultSetMetaData, ResultS
         printf(formatValue, "null");
         handleException(e);
       }
-    } else {
+    } 
+    else if (align) {
       if (i == 2 && resultSetMetaData.getColumnName(2).equals(GROUPBY_DEVICE_COLUMN_NAME)) {
         printf("%" + deviceColumnLength + "s|", res.getString(i));
       } else {
         printf(formatValue, res.getString(i));
       }
     }
+    // for disable align clause
+    else {
+      if (res.getString(i * 2 - 2) == null) {
 
 Review comment:
   These "2"s appear frequently, I suggest replace them with a constant and add a comment for the constant to explain what column is skipped.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] sonarcloud[bot] removed a comment on issue #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on issue #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#issuecomment-574465051
 
 
   Kudos, SonarCloud Quality Gate passed!
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) (and [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) to review)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [12 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo.png' alt='No Coverage information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738) No Coverage information  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list)
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366226547
 
 

 ##########
 File path: jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBNonAlignQueryResultSet.java
 ##########
 @@ -0,0 +1,1289 @@
+/*
+ * 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.jdbc;
+
+import org.apache.iotdb.rpc.IoTDBRPCException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.service.rpc.thrift.*;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.thrift.TException;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.sql.Date;
+import java.sql.*;
+import java.util.*;
+
+public class IoTDBNonAlignQueryResultSet implements ResultSet {
+
+  private static final String TIMESTAMP_STR = "Time";
+  private static final int TIMESTAMP_STR_LENGTH = 4;
+  private static final int START_INDEX = 2;
+  private static final String VALUE_IS_NULL = "The value got by %s (column name) is NULL.";
+  private static final String EMPTY_STR = "";
+  private Statement statement = null;
+  private String sql;
+  private SQLWarning warningChain = null;
+  private boolean isClosed = false;
+  private TSIService.Iface client = null;
+  private List<String> columnInfoList; // no deduplication
 
 Review comment:
   "Info" is too ambiguous, just call it "columnName".

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] JackieTien97 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366322249
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/NonAlignEngineDataSet.java
 ##########
 @@ -0,0 +1,354 @@
+/*
+ * 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.dataset;
+
+import org.apache.iotdb.db.query.pool.QueryTaskPoolManager;
+import org.apache.iotdb.db.query.reader.ManagedSeriesReader;
+import org.apache.iotdb.db.tools.watermark.WatermarkEncoder;
+import org.apache.iotdb.service.rpc.thrift.TSQueryNonAlignDataSet;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicIntegerArray;
+
+public class NonAlignEngineDataSet extends QueryDataSet {
+
+  private static class ReadTask implements Runnable {
+
+    private final ManagedSeriesReader reader;
+    private BlockingQueue<Pair<ByteBuffer, ByteBuffer>> blockingQueue;
+    private WatermarkEncoder encoder;
+    NonAlignEngineDataSet dataSet;
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] sonarcloud[bot] commented on issue #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on issue #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#issuecomment-574130028
 
 
   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C.png' alt='C' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) (and [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) to review)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [15 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo.png' alt='No Coverage information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738) No Coverage information  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/20plus.png' alt='46.7%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list) [46.7% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list)
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on issue #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on issue #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#issuecomment-574168364
 
 
   CodeA does not encounter any concurrent correctness while CodeB does, so it is reasonable to believe the usage of primitives is safe.
   CodeA:
   ```
   public static void main(String[] args) throws InterruptedException {
       int[] ints = new int[100];
       ExecutorService executorService = Executors.newFixedThreadPool(10);
       for (int i = 0; i < 100; i++) {
         executorService.submit(new TestRunnable(i, executorService, ints));
       }
       Thread.sleep(10000);
       executorService.shutdownNow();
       for (int anInt : ints) {
         System.out.println(anInt);
       }
     }
   
     static class TestRunnable implements Runnable {
   
       int i;
       volatile int remain = 100;
       ExecutorService service;
       int[] ints;
   
       public TestRunnable(int i, ExecutorService service, int[] ints) {
         this.i = i;
         this.service = service;
         this.ints = ints;
       }
   
       @Override
       public void run() {
         ints[i] ++;
         remain --;
         if (remain > 0) {
           service.submit(this);
         }
       }
     }
   ```
   
   CodeB:
   ```
   public static void main(String[] args) throws InterruptedException {
       int[] ints = new int[100];
       ExecutorService executorService = Executors.newFixedThreadPool(10);
       for (int i = 0; i < 100; i++) {
         executorService.submit(new TestRunnable(i, executorService, ints));
       }
       Thread.sleep(10000);
       executorService.shutdownNow();
       for (int anInt : ints) {
         System.out.println(anInt);
       }
     }
   
     static class TestRunnable implements Runnable {
   
       int i;
       volatile int remain = 100;
       ExecutorService service;
       int[] ints;
   
       public TestRunnable(int i, ExecutorService service, int[] ints) {
         this.i = i;
         this.service = service;
         this.ints = ints;
       }
   
       @Override
       public void run() {
         ints[0] ++;
         remain --;
         if (remain > 0) {
           service.submit(this);
         }
       }
     }
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] JackieTien97 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366322285
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/NonAlignEngineDataSet.java
 ##########
 @@ -0,0 +1,354 @@
+/*
+ * 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.dataset;
+
+import org.apache.iotdb.db.query.pool.QueryTaskPoolManager;
+import org.apache.iotdb.db.query.reader.ManagedSeriesReader;
+import org.apache.iotdb.db.tools.watermark.WatermarkEncoder;
+import org.apache.iotdb.service.rpc.thrift.TSQueryNonAlignDataSet;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicIntegerArray;
+
+public class NonAlignEngineDataSet extends QueryDataSet {
+
+  private static class ReadTask implements Runnable {
+
+    private final ManagedSeriesReader reader;
+    private BlockingQueue<Pair<ByteBuffer, ByteBuffer>> blockingQueue;
+    private WatermarkEncoder encoder;
+    NonAlignEngineDataSet dataSet;
+    private int index;
+
+
+    public ReadTask(ManagedSeriesReader reader,
+                    BlockingQueue<Pair<ByteBuffer, ByteBuffer>> blockingQueue,
+                    WatermarkEncoder encoder, NonAlignEngineDataSet dataSet, int index) {
+      this.reader = reader;
+      this.blockingQueue = blockingQueue;
+      this.encoder = encoder;
+      this.dataSet = dataSet;
+      this.index = index;
+    }
+
+    @Override
+    public void run() {
+      PublicBAOS timeBAOS = new PublicBAOS();
+      PublicBAOS valueBAOS = new PublicBAOS();
+      try {
+        synchronized (reader) {
+          // if the task is submitted, there must be free space in the queue
+          // so here we don't need to check whether the queue has free space
+          // the reader has next batch
+          if ((dataSet.cachedBatchData[index] != null && dataSet.cachedBatchData[index].hasCurrent())
+                  || reader.hasNextBatch()) {
+            BatchData batchData;
+            if (dataSet.cachedBatchData[index] != null && dataSet.cachedBatchData[index].hasCurrent())
+              batchData = dataSet.cachedBatchData[index];
+            else
+              batchData = reader.nextBatch();
+
+            int rowCount = 0;
+            while (rowCount < dataSet.fetchSize) {
+
+              if ((dataSet.limit > 0 && dataSet.alreadyReturnedRowNumArray.get(index) >= dataSet.limit)) {
+                break;
+              }
+
+              if (batchData != null && batchData.hasCurrent()) {
+                if (dataSet.offsetArray.get(index) == 0) {
+                  long time = batchData.currentTime();
+                  ReadWriteIOUtils.write(time, timeBAOS);
+                  TSDataType type = batchData.getDataType();
+                  switch (type) {
+                    case INT32:
+                      int intValue = batchData.getInt();
+                      if (encoder != null && encoder.needEncode(time)) {
+                        intValue = encoder.encodeInt(intValue, time);
+                      }
+                      ReadWriteIOUtils.write(intValue, valueBAOS);
+                      break;
+                    case INT64:
+                      long longValue = batchData.getLong();
+                      if (encoder != null && encoder.needEncode(time)) {
+                        longValue = encoder.encodeLong(longValue, time);
+                      }
+                      ReadWriteIOUtils.write(longValue, valueBAOS);
+                      break;
+                    case FLOAT:
+                      float floatValue = batchData.getFloat();
+                      if (encoder != null && encoder.needEncode(time)) {
+                        floatValue = encoder.encodeFloat(floatValue, time);
+                      }
+                      ReadWriteIOUtils.write(floatValue, valueBAOS);
+                      break;
+                    case DOUBLE:
+                      double doubleValue = batchData.getDouble();
+                      if (encoder != null && encoder.needEncode(time)) {
+                        doubleValue = encoder.encodeDouble(doubleValue, time);
+                      }
+                      ReadWriteIOUtils.write(doubleValue, valueBAOS);
+                      break;
+                    case BOOLEAN:
+                      ReadWriteIOUtils.write(batchData.getBoolean(),
+                              valueBAOS);
+                      break;
+                    case TEXT:
+                      ReadWriteIOUtils
+                              .write(batchData.getBinary(),
+                                      valueBAOS);
+                      break;
+                    default:
+                      throw new UnSupportedDataTypeException(
+                              String.format("Data type %s is not supported.", type));
+                  }
+                }
+                batchData.next();
+              }
+              else {
+                if (reader.hasNextBatch()) {
+                  batchData = reader.nextBatch();
+                  dataSet.cachedBatchData[index] = batchData;
+                  continue;
+                }
+                else
+                  break;
+              }
+              if (dataSet.offsetArray.get(index) == 0) {
+                rowCount++;
+                if (dataSet.limit > 0) {
+                  dataSet.alreadyReturnedRowNumArray.incrementAndGet(index);
+                }
+              } else {
+                dataSet.offsetArray.decrementAndGet(index);
+              }
+            }
+            if (rowCount == 0) {
+              blockingQueue.put(new Pair(null, null));
+              // set the hasRemaining field in reader to false
+              // tell the Consumer not to submit another task for this reader any more
+              reader.setHasRemaining(false);
+              // remove itself from the QueryTaskPoolManager
+              reader.setManagedByQueryManager(false);
+              return;
+            }
+
+            ByteBuffer timeBuffer = ByteBuffer.allocate(timeBAOS.size());
+            timeBuffer.put(timeBAOS.getBuf(), 0, timeBAOS.size());
+            timeBuffer.flip();
+            ByteBuffer valueBuffer = ByteBuffer.allocate(valueBAOS.size());
+            valueBuffer.put(valueBAOS.getBuf(), 0, valueBAOS.size());
+            valueBuffer.flip();
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin merged pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
qiaojialin merged pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] sonarcloud[bot] commented on issue #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on issue #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#issuecomment-574155814
 
 
   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) (and [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) to review)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [15 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo.png' alt='No Coverage information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738) No Coverage information  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/20plus.png' alt='46.6%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list) [46.6% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list)
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366225892
 
 

 ##########
 File path: jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBNonAlignQueryResultSet.java
 ##########
 @@ -0,0 +1,1289 @@
+/*
+ * 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.jdbc;
+
+import org.apache.iotdb.rpc.IoTDBRPCException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.service.rpc.thrift.*;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.thrift.TException;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.sql.Date;
+import java.sql.*;
+import java.util.*;
+
+public class IoTDBNonAlignQueryResultSet implements ResultSet {
+
+  private static final String TIMESTAMP_STR = "Time";
+  private static final int TIMESTAMP_STR_LENGTH = 4;
+  private static final int START_INDEX = 2;
+  private static final String VALUE_IS_NULL = "The value got by %s (column name) is NULL.";
+  private static final String EMPTY_STR = "";
+  private Statement statement = null;
+  private String sql;
+  private SQLWarning warningChain = null;
+  private boolean isClosed = false;
+  private TSIService.Iface client = null;
+  private List<String> columnInfoList; // no deduplication
+  private List<String> columnTypeList; // no deduplication
+  private Map<String, Integer> columnInfoMap; // used because the server returns deduplicated columns
+  private List<String> columnTypeDeduplicatedList; // deduplicated from columnTypeList
 
 Review comment:
   It would be better if you just make it a List of TsDataType, it is unnecessary to convert a string to TsDataType every time.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366276562
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/NonAlignEngineDataSet.java
 ##########
 @@ -0,0 +1,354 @@
+/*
+ * 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.dataset;
+
+import org.apache.iotdb.db.query.pool.QueryTaskPoolManager;
+import org.apache.iotdb.db.query.reader.ManagedSeriesReader;
+import org.apache.iotdb.db.tools.watermark.WatermarkEncoder;
+import org.apache.iotdb.service.rpc.thrift.TSQueryNonAlignDataSet;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicIntegerArray;
+
+public class NonAlignEngineDataSet extends QueryDataSet {
+
+  private static class ReadTask implements Runnable {
+
+    private final ManagedSeriesReader reader;
+    private BlockingQueue<Pair<ByteBuffer, ByteBuffer>> blockingQueue;
+    private WatermarkEncoder encoder;
+    NonAlignEngineDataSet dataSet;
+    private int index;
+
+
+    public ReadTask(ManagedSeriesReader reader,
+                    BlockingQueue<Pair<ByteBuffer, ByteBuffer>> blockingQueue,
+                    WatermarkEncoder encoder, NonAlignEngineDataSet dataSet, int index) {
+      this.reader = reader;
+      this.blockingQueue = blockingQueue;
+      this.encoder = encoder;
+      this.dataSet = dataSet;
+      this.index = index;
+    }
+
+    @Override
+    public void run() {
+      PublicBAOS timeBAOS = new PublicBAOS();
+      PublicBAOS valueBAOS = new PublicBAOS();
+      try {
+        synchronized (reader) {
+          // if the task is submitted, there must be free space in the queue
+          // so here we don't need to check whether the queue has free space
+          // the reader has next batch
+          if ((dataSet.cachedBatchData[index] != null && dataSet.cachedBatchData[index].hasCurrent())
+                  || reader.hasNextBatch()) {
+            BatchData batchData;
+            if (dataSet.cachedBatchData[index] != null && dataSet.cachedBatchData[index].hasCurrent())
+              batchData = dataSet.cachedBatchData[index];
+            else
+              batchData = reader.nextBatch();
 
 Review comment:
   Always use "{}" for code blocks (even if it only contains a single line), as we obey the google code-style.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] sonarcloud[bot] commented on issue #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on issue #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#issuecomment-574465051
 
 
   Kudos, SonarCloud Quality Gate passed!
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) (and [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) to review)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [12 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo.png' alt='No Coverage information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738) No Coverage information  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list)
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] sonarcloud[bot] removed a comment on issue #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on issue #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#issuecomment-574155814
 
 
   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) (and [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) to review)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [15 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo.png' alt='No Coverage information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738) No Coverage information  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/20plus.png' alt='46.6%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list) [46.6% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list)
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] JackieTien97 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366325201
 
 

 ##########
 File path: jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBNonAlignQueryResultSet.java
 ##########
 @@ -0,0 +1,1289 @@
+/*
+ * 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.jdbc;
+
+import org.apache.iotdb.rpc.IoTDBRPCException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.service.rpc.thrift.*;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.thrift.TException;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.sql.Date;
+import java.sql.*;
+import java.util.*;
+
+public class IoTDBNonAlignQueryResultSet implements ResultSet {
+
+  private static final String TIMESTAMP_STR = "Time";
+  private static final int TIMESTAMP_STR_LENGTH = 4;
+  private static final int START_INDEX = 2;
+  private static final String VALUE_IS_NULL = "The value got by %s (column name) is NULL.";
+  private static final String EMPTY_STR = "";
+  private Statement statement = null;
+  private String sql;
+  private SQLWarning warningChain = null;
+  private boolean isClosed = false;
+  private TSIService.Iface client = null;
+  private List<String> columnInfoList; // no deduplication
+  private List<String> columnTypeList; // no deduplication
+  private Map<String, Integer> columnInfoMap; // used because the server returns deduplicated columns
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366224122
 
 

 ##########
 File path: example/jdbc/src/main/java/org/apache/iotdb/JDBCExample.java
 ##########
 @@ -1,21 +1,3 @@
-/*
- * 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;
 
 Review comment:
   Why remove the license?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366224728
 
 

 ##########
 File path: example/jdbc/src/main/java/org/apache/iotdb/JDBCExample.java
 ##########
 @@ -31,23 +13,11 @@ public static void main(String[] args) throws ClassNotFoundException, SQLExcepti
     Class.forName("org.apache.iotdb.jdbc.IoTDBDriver");
     try (Connection connection = DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
          Statement statement = connection.createStatement()) {
-      statement.execute("SET STORAGE GROUP TO root.sg1");
-      statement.execute("CREATE TIMESERIES root.sg1.d1.s1 WITH DATATYPE=INT64, ENCODING=RLE");
-      statement.execute("CREATE TIMESERIES root.sg1.d1.s2 WITH DATATYPE=INT64, ENCODING=RLE");
-      statement.execute("CREATE TIMESERIES root.sg1.d1.s3 WITH DATATYPE=INT64, ENCODING=RLE");
-
-      for (int i = 0; i <= 100; i++) {
-        statement.addBatch("insert into root.sg1.d1(timestamp, s1, s2, s3) values("+ i + "," + 1 + "," + 1 + "," + 1 + ")");
-      }
-      statement.executeBatch();
-      statement.clearBatch();
-
-      ResultSet resultSet = statement.executeQuery("select * from root where time <= 10");
-      outputResult(resultSet);
-      resultSet = statement.executeQuery("select count(*) from root");
-      outputResult(resultSet);
-      resultSet = statement.executeQuery("select count(*) from root where time >= 1 and time <= 100 group by ([0, 100], 20ms, 20ms)");
+      long startTime = System.currentTimeMillis();
+      ResultSet resultSet = statement.executeQuery("select * from root where time < 100000000 disable align");
       outputResult(resultSet);
+      long endTime = System.currentTimeMillis();
+      System.out.println("Cost Time: " + (endTime - startTime));
 
 Review comment:
   I don't think you should remove the old examples unless they are incorrect, you should only add the new ones.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] JackieTien97 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366322207
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/NonAlignEngineDataSet.java
 ##########
 @@ -0,0 +1,354 @@
+/*
+ * 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.dataset;
+
+import org.apache.iotdb.db.query.pool.QueryTaskPoolManager;
+import org.apache.iotdb.db.query.reader.ManagedSeriesReader;
+import org.apache.iotdb.db.tools.watermark.WatermarkEncoder;
+import org.apache.iotdb.service.rpc.thrift.TSQueryNonAlignDataSet;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicIntegerArray;
+
+public class NonAlignEngineDataSet extends QueryDataSet {
+
+  private static class ReadTask implements Runnable {
+
+    private final ManagedSeriesReader reader;
+    private BlockingQueue<Pair<ByteBuffer, ByteBuffer>> blockingQueue;
+    private WatermarkEncoder encoder;
+    NonAlignEngineDataSet dataSet;
+    private int index;
+
+
+    public ReadTask(ManagedSeriesReader reader,
+                    BlockingQueue<Pair<ByteBuffer, ByteBuffer>> blockingQueue,
+                    WatermarkEncoder encoder, NonAlignEngineDataSet dataSet, int index) {
+      this.reader = reader;
+      this.blockingQueue = blockingQueue;
+      this.encoder = encoder;
+      this.dataSet = dataSet;
+      this.index = index;
+    }
+
+    @Override
+    public void run() {
+      PublicBAOS timeBAOS = new PublicBAOS();
+      PublicBAOS valueBAOS = new PublicBAOS();
+      try {
+        synchronized (reader) {
+          // if the task is submitted, there must be free space in the queue
+          // so here we don't need to check whether the queue has free space
+          // the reader has next batch
+          if ((dataSet.cachedBatchData[index] != null && dataSet.cachedBatchData[index].hasCurrent())
+                  || reader.hasNextBatch()) {
+            BatchData batchData;
+            if (dataSet.cachedBatchData[index] != null && dataSet.cachedBatchData[index].hasCurrent())
+              batchData = dataSet.cachedBatchData[index];
+            else
+              batchData = reader.nextBatch();
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] sonarcloud[bot] commented on issue #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on issue #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#issuecomment-574180330
 
 
   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) (and [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) to review)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [14 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo.png' alt='No Coverage information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738) No Coverage information  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/20plus.png' alt='43.5%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list) [43.5% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list)
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366218327
 
 

 ##########
 File path: client/src/main/java/org/apache/iotdb/client/AbstractClient.java
 ##########
 @@ -261,28 +270,46 @@ private static void printShow(int colCount, ResultSet res) throws SQLException {
     println();
   }
 
-  private static void printRowData(boolean printTimestamp, ResultSet res, ZoneId zoneId,
+  private static void printRowData(boolean printTimestamp, boolean align, ResultSet res, ZoneId zoneId,
       ResultSetMetaData resultSetMetaData, int colCount)
       throws SQLException {
     if (displayCnt < maxPrintRowCount) { // NOTE displayCnt only works on queried data results
       print("|");
-      if (printTimestamp) {
-        printf(formatTime, formatDatetime(res.getLong(TIMESTAMP_STR), zoneId));
-        for (int i = 2; i <= colCount; i++) {
-          printColumnData(resultSetMetaData, res, i, zoneId);
+      if (align) {
+        if (printTimestamp) {
+          printf(formatTime, formatDatetime(res.getLong(TIMESTAMP_STR), zoneId));
+          for (int i = 2; i <= colCount; i++) {
+            printColumnData(resultSetMetaData, align, res, i, zoneId);
+          }
+        } else {
+          for (int i = 1; i <= colCount; i++) {
+            printColumnData(resultSetMetaData, align, res, i, zoneId);
+          }
         }
-      } else {
-        for (int i = 1; i <= colCount; i++) {
-          printf(formatValue, res.getString(i));
+      }
+      else {
+        for (int i = 2; i <= colCount / 2 + 1; i++) {
+          if (printTimestamp) {
+            // timeLabel used for indicating the time column.
+            String timeLabel = TIMESTAMP_STR + resultSetMetaData.getColumnLabel(2 * i - 2);
+            if (res.getLong(timeLabel) == 0) {
+              // blank space
+              printf(formatTime, "");
 
 Review comment:
   If the timestamp is truly a "0", will it be translated into space? If so, I don't think it is reasonable.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] JackieTien97 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366325254
 
 

 ##########
 File path: jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBNonAlignQueryResultSet.java
 ##########
 @@ -0,0 +1,1289 @@
+/*
+ * 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.jdbc;
+
+import org.apache.iotdb.rpc.IoTDBRPCException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.service.rpc.thrift.*;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.thrift.TException;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.sql.Date;
+import java.sql.*;
+import java.util.*;
+
+public class IoTDBNonAlignQueryResultSet implements ResultSet {
+
+  private static final String TIMESTAMP_STR = "Time";
+  private static final int TIMESTAMP_STR_LENGTH = 4;
+  private static final int START_INDEX = 2;
+  private static final String VALUE_IS_NULL = "The value got by %s (column name) is NULL.";
+  private static final String EMPTY_STR = "";
+  private Statement statement = null;
+  private String sql;
+  private SQLWarning warningChain = null;
+  private boolean isClosed = false;
+  private TSIService.Iface client = null;
+  private List<String> columnInfoList; // no deduplication
+  private List<String> columnTypeList; // no deduplication
+  private Map<String, Integer> columnInfoMap; // used because the server returns deduplicated columns
+  private List<String> columnTypeDeduplicatedList; // deduplicated from columnTypeList
+  private int fetchSize;
+  private boolean emptyResultSet = false;
+
+  private TSQueryNonAlignDataSet tsQueryNonAlignDataSet = null;
+  private byte[][] times; // used for disable align
+  private byte[][] values; // used to cache the current row record value
+
+  private long sessionId;
+  private long queryId;
+  private boolean ignoreTimeStamp = false;
+
+  public IoTDBNonAlignQueryResultSet() {
+    // do nothing
+  }
+
+  // for disable align clause
+  public IoTDBNonAlignQueryResultSet(Statement statement, List<String> columnNameList,
+                                     List<String> columnTypeList, boolean ignoreTimeStamp, TSIService.Iface client,
+                                     String sql, long queryId, long sessionId, TSQueryNonAlignDataSet dataset)
+          throws SQLException {
+    this.statement = statement;
+    this.fetchSize = statement.getFetchSize();
+    this.columnTypeList = columnTypeList;
+
+    times = new byte[columnNameList.size()][Long.BYTES];
+    values = new byte[columnNameList.size()][];
+
+    this.columnInfoList = new ArrayList<>();
+    // deduplicate and map
+    this.columnInfoMap = new HashMap<>();
+    this.columnInfoMap.put(TIMESTAMP_STR, 1);
+    this.columnTypeDeduplicatedList = new ArrayList<>();
+    int index = START_INDEX;
+    for (int i = 0; i < columnNameList.size(); i++) {
+      String name = columnNameList.get(i);
+      columnInfoList.add(TIMESTAMP_STR + name);
+      columnInfoList.add(name);
+      if (!columnInfoMap.containsKey(name)) {
+        columnInfoMap.put(name, index++);
+        columnTypeDeduplicatedList.add(columnTypeList.get(i));
+      }
+    }
+
+    this.ignoreTimeStamp = ignoreTimeStamp;
+    this.client = client;
+    this.sql = sql;
+    this.queryId = queryId;
+    this.tsQueryNonAlignDataSet = dataset;
+    this.sessionId = sessionId;
+  }
+
+  @Override
+  public boolean isWrapperFor(Class<?> iface) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public <T> T unwrap(Class<T> iface) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean absolute(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void afterLast() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void beforeFirst() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void cancelRowUpdates() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void clearWarnings() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void close() throws SQLException {
+    if (isClosed) {
+      return;
+    }
+    if (client != null) {
+      try {
+        TSCloseOperationReq closeReq = new TSCloseOperationReq(sessionId);
+        closeReq.setQueryId(queryId);
+        TSStatus closeResp = client.closeOperation(closeReq);
+        RpcUtils.verifySuccess(closeResp);
+      } catch (IoTDBRPCException e) {
+        throw new SQLException("Error occurs for close opeation in server side becasuse " + e);
+      } catch (TException e) {
+        throw new SQLException(
+                "Error occurs when connecting to server for close operation, becasue: " + e);
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366230138
 
 

 ##########
 File path: jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBNonAlignQueryResultSet.java
 ##########
 @@ -0,0 +1,1289 @@
+/*
+ * 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.jdbc;
+
+import org.apache.iotdb.rpc.IoTDBRPCException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.service.rpc.thrift.*;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.thrift.TException;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.sql.Date;
+import java.sql.*;
+import java.util.*;
+
+public class IoTDBNonAlignQueryResultSet implements ResultSet {
+
+  private static final String TIMESTAMP_STR = "Time";
+  private static final int TIMESTAMP_STR_LENGTH = 4;
+  private static final int START_INDEX = 2;
+  private static final String VALUE_IS_NULL = "The value got by %s (column name) is NULL.";
+  private static final String EMPTY_STR = "";
+  private Statement statement = null;
+  private String sql;
+  private SQLWarning warningChain = null;
+  private boolean isClosed = false;
+  private TSIService.Iface client = null;
+  private List<String> columnInfoList; // no deduplication
+  private List<String> columnTypeList; // no deduplication
+  private Map<String, Integer> columnInfoMap; // used because the server returns deduplicated columns
+  private List<String> columnTypeDeduplicatedList; // deduplicated from columnTypeList
+  private int fetchSize;
+  private boolean emptyResultSet = false;
+
+  private TSQueryNonAlignDataSet tsQueryNonAlignDataSet = null;
+  private byte[][] times; // used for disable align
+  private byte[][] values; // used to cache the current row record value
+
+  private long sessionId;
+  private long queryId;
+  private boolean ignoreTimeStamp = false;
+
+  public IoTDBNonAlignQueryResultSet() {
+    // do nothing
+  }
+
+  // for disable align clause
+  public IoTDBNonAlignQueryResultSet(Statement statement, List<String> columnNameList,
+                                     List<String> columnTypeList, boolean ignoreTimeStamp, TSIService.Iface client,
+                                     String sql, long queryId, long sessionId, TSQueryNonAlignDataSet dataset)
+          throws SQLException {
+    this.statement = statement;
+    this.fetchSize = statement.getFetchSize();
+    this.columnTypeList = columnTypeList;
+
+    times = new byte[columnNameList.size()][Long.BYTES];
+    values = new byte[columnNameList.size()][];
+
+    this.columnInfoList = new ArrayList<>();
+    // deduplicate and map
+    this.columnInfoMap = new HashMap<>();
+    this.columnInfoMap.put(TIMESTAMP_STR, 1);
+    this.columnTypeDeduplicatedList = new ArrayList<>();
+    int index = START_INDEX;
+    for (int i = 0; i < columnNameList.size(); i++) {
+      String name = columnNameList.get(i);
+      columnInfoList.add(TIMESTAMP_STR + name);
+      columnInfoList.add(name);
+      if (!columnInfoMap.containsKey(name)) {
+        columnInfoMap.put(name, index++);
+        columnTypeDeduplicatedList.add(columnTypeList.get(i));
+      }
+    }
+
+    this.ignoreTimeStamp = ignoreTimeStamp;
+    this.client = client;
+    this.sql = sql;
+    this.queryId = queryId;
+    this.tsQueryNonAlignDataSet = dataset;
+    this.sessionId = sessionId;
+  }
+
+  @Override
+  public boolean isWrapperFor(Class<?> iface) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public <T> T unwrap(Class<T> iface) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean absolute(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void afterLast() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void beforeFirst() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void cancelRowUpdates() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void clearWarnings() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void close() throws SQLException {
+    if (isClosed) {
+      return;
+    }
+    if (client != null) {
+      try {
+        TSCloseOperationReq closeReq = new TSCloseOperationReq(sessionId);
+        closeReq.setQueryId(queryId);
+        TSStatus closeResp = client.closeOperation(closeReq);
+        RpcUtils.verifySuccess(closeResp);
+      } catch (IoTDBRPCException e) {
+        throw new SQLException("Error occurs for close opeation in server side becasuse " + e);
+      } catch (TException e) {
+        throw new SQLException(
+                "Error occurs when connecting to server for close operation, becasue: " + e);
+      }
+    }
+    client = null;
+    isClosed = true;
+  }
+
+
+  @Override
+  public void deleteRow() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int findColumn(String columnName) {
+    return columnInfoMap.get(columnName);
+  }
+
+  @Override
+  public boolean first() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
 
 Review comment:
   These methods are duplicated with the aligned one, better to extract them into a base class like "IoTDBResultSet".

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on issue #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on issue #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#issuecomment-574451205
 
 
   Great, but please see the code smells and try resolving them as much as possible.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] sonarcloud[bot] removed a comment on issue #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on issue #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#issuecomment-574180330
 
 
   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) (and [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) to review)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [14 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo.png' alt='No Coverage information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738) No Coverage information  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/20plus.png' alt='43.5%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list) [43.5% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list)
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] JackieTien97 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366325171
 
 

 ##########
 File path: jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBNonAlignQueryResultSet.java
 ##########
 @@ -0,0 +1,1289 @@
+/*
+ * 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.jdbc;
+
+import org.apache.iotdb.rpc.IoTDBRPCException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.service.rpc.thrift.*;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.thrift.TException;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.sql.Date;
+import java.sql.*;
+import java.util.*;
+
+public class IoTDBNonAlignQueryResultSet implements ResultSet {
+
+  private static final String TIMESTAMP_STR = "Time";
+  private static final int TIMESTAMP_STR_LENGTH = 4;
+  private static final int START_INDEX = 2;
+  private static final String VALUE_IS_NULL = "The value got by %s (column name) is NULL.";
+  private static final String EMPTY_STR = "";
+  private Statement statement = null;
+  private String sql;
+  private SQLWarning warningChain = null;
+  private boolean isClosed = false;
+  private TSIService.Iface client = null;
+  private List<String> columnInfoList; // no deduplication
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 edited a comment on issue #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 edited a comment on issue #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#issuecomment-574168364
 
 
   CodeA does not encounter any concurrent correctness problem while CodeB does, so it is reasonable to believe the usage of primitives is safe.
   CodeA:
   ```
   public static void main(String[] args) throws InterruptedException {
       int[] ints = new int[100];
       ExecutorService executorService = Executors.newFixedThreadPool(10);
       for (int i = 0; i < 100; i++) {
         executorService.submit(new TestRunnable(i, executorService, ints));
       }
       Thread.sleep(10000);
       executorService.shutdownNow();
       for (int anInt : ints) {
         System.out.println(anInt);
       }
     }
   
     static class TestRunnable implements Runnable {
   
       int i;
       volatile int remain = 100;
       ExecutorService service;
       int[] ints;
   
       public TestRunnable(int i, ExecutorService service, int[] ints) {
         this.i = i;
         this.service = service;
         this.ints = ints;
       }
   
       @Override
       public void run() {
         ints[i] ++;
         remain --;
         if (remain > 0) {
           service.submit(this);
         }
       }
     }
   ```
   
   CodeB:
   ```
   public static void main(String[] args) throws InterruptedException {
       int[] ints = new int[100];
       ExecutorService executorService = Executors.newFixedThreadPool(10);
       for (int i = 0; i < 100; i++) {
         executorService.submit(new TestRunnable(i, executorService, ints));
       }
       Thread.sleep(10000);
       executorService.shutdownNow();
       for (int anInt : ints) {
         System.out.println(anInt);
       }
     }
   
     static class TestRunnable implements Runnable {
   
       int i;
       volatile int remain = 100;
       ExecutorService service;
       int[] ints;
   
       public TestRunnable(int i, ExecutorService service, int[] ints) {
         this.i = i;
         this.service = service;
         this.ints = ints;
       }
   
       @Override
       public void run() {
         ints[0] ++;
         remain --;
         if (remain > 0) {
           service.submit(this);
         }
       }
     }
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366229362
 
 

 ##########
 File path: jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBNonAlignQueryResultSet.java
 ##########
 @@ -0,0 +1,1289 @@
+/*
+ * 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.jdbc;
+
+import org.apache.iotdb.rpc.IoTDBRPCException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.service.rpc.thrift.*;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.thrift.TException;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.sql.Date;
+import java.sql.*;
+import java.util.*;
+
+public class IoTDBNonAlignQueryResultSet implements ResultSet {
+
+  private static final String TIMESTAMP_STR = "Time";
+  private static final int TIMESTAMP_STR_LENGTH = 4;
+  private static final int START_INDEX = 2;
+  private static final String VALUE_IS_NULL = "The value got by %s (column name) is NULL.";
+  private static final String EMPTY_STR = "";
+  private Statement statement = null;
+  private String sql;
+  private SQLWarning warningChain = null;
+  private boolean isClosed = false;
+  private TSIService.Iface client = null;
+  private List<String> columnInfoList; // no deduplication
+  private List<String> columnTypeList; // no deduplication
+  private Map<String, Integer> columnInfoMap; // used because the server returns deduplicated columns
+  private List<String> columnTypeDeduplicatedList; // deduplicated from columnTypeList
+  private int fetchSize;
+  private boolean emptyResultSet = false;
+
+  private TSQueryNonAlignDataSet tsQueryNonAlignDataSet = null;
+  private byte[][] times; // used for disable align
+  private byte[][] values; // used to cache the current row record value
+
+  private long sessionId;
+  private long queryId;
+  private boolean ignoreTimeStamp = false;
+
+  public IoTDBNonAlignQueryResultSet() {
+    // do nothing
+  }
+
+  // for disable align clause
+  public IoTDBNonAlignQueryResultSet(Statement statement, List<String> columnNameList,
+                                     List<String> columnTypeList, boolean ignoreTimeStamp, TSIService.Iface client,
+                                     String sql, long queryId, long sessionId, TSQueryNonAlignDataSet dataset)
+          throws SQLException {
+    this.statement = statement;
+    this.fetchSize = statement.getFetchSize();
+    this.columnTypeList = columnTypeList;
+
+    times = new byte[columnNameList.size()][Long.BYTES];
+    values = new byte[columnNameList.size()][];
+
+    this.columnInfoList = new ArrayList<>();
+    // deduplicate and map
+    this.columnInfoMap = new HashMap<>();
+    this.columnInfoMap.put(TIMESTAMP_STR, 1);
+    this.columnTypeDeduplicatedList = new ArrayList<>();
+    int index = START_INDEX;
+    for (int i = 0; i < columnNameList.size(); i++) {
+      String name = columnNameList.get(i);
+      columnInfoList.add(TIMESTAMP_STR + name);
+      columnInfoList.add(name);
+      if (!columnInfoMap.containsKey(name)) {
+        columnInfoMap.put(name, index++);
+        columnTypeDeduplicatedList.add(columnTypeList.get(i));
+      }
+    }
+
+    this.ignoreTimeStamp = ignoreTimeStamp;
+    this.client = client;
+    this.sql = sql;
+    this.queryId = queryId;
+    this.tsQueryNonAlignDataSet = dataset;
+    this.sessionId = sessionId;
+  }
+
+  @Override
+  public boolean isWrapperFor(Class<?> iface) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public <T> T unwrap(Class<T> iface) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean absolute(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void afterLast() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void beforeFirst() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void cancelRowUpdates() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void clearWarnings() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void close() throws SQLException {
+    if (isClosed) {
+      return;
+    }
+    if (client != null) {
+      try {
+        TSCloseOperationReq closeReq = new TSCloseOperationReq(sessionId);
+        closeReq.setQueryId(queryId);
+        TSStatus closeResp = client.closeOperation(closeReq);
+        RpcUtils.verifySuccess(closeResp);
+      } catch (IoTDBRPCException e) {
+        throw new SQLException("Error occurs for close opeation in server side becasuse " + e);
+      } catch (TException e) {
+        throw new SQLException(
+                "Error occurs when connecting to server for close operation, becasue: " + e);
 
 Review comment:
   Always keep the stack trace unless you are sure the exception is minor.
   ```
   throw new SQLException("Error occurs when connecting to server for close operation ", e);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366296983
 
 

 ##########
 File path: service-rpc/rpc-changelist.md
 ##########
 @@ -107,9 +107,11 @@ Last Updated on November 12th, 2019 by Tian Jiang.
 | Latest Changes                                               | Related Committers                 |
 | ------------------------------------------------------------ | ---------------------------------- |
 | Add parameter sessionId in getTimeZone, getProperties, setStorageGroup, createTimeseries... | Tian Jiang|
+| Add struct TSQueryNonAlignDataSet                            | Haonan Hou|
 
 ## 3. Update
 
 | Latest Changes                                               | Related Committers     |
 | ------------------------------------------------------------ | ---------------------- |
-| Replace TS_SessionHandles with SessionIds, TSOperationHandle with queryIds  | Tian Jiang  |
\ No newline at end of file
+| Replace TS_SessionHandles with SessionIds, TSOperationHandle with queryIds  | Tian Jiang  |
+| Add optional TSQueryNonAlignDataSet in TSExecuteStatementResp, TSFetchResultsResp  | Haonan Hou |
 
 Review comment:
   And the field "isAlign".

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366273177
 
 

 ##########
 File path: jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBNonAlignQueryResultSet.java
 ##########
 @@ -0,0 +1,1289 @@
+/*
+ * 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.jdbc;
+
+import org.apache.iotdb.rpc.IoTDBRPCException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.service.rpc.thrift.*;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.thrift.TException;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.sql.Date;
+import java.sql.*;
+import java.util.*;
+
+public class IoTDBNonAlignQueryResultSet implements ResultSet {
+
+  private static final String TIMESTAMP_STR = "Time";
+  private static final int TIMESTAMP_STR_LENGTH = 4;
+  private static final int START_INDEX = 2;
+  private static final String VALUE_IS_NULL = "The value got by %s (column name) is NULL.";
+  private static final String EMPTY_STR = "";
+  private Statement statement = null;
+  private String sql;
+  private SQLWarning warningChain = null;
+  private boolean isClosed = false;
+  private TSIService.Iface client = null;
+  private List<String> columnInfoList; // no deduplication
+  private List<String> columnTypeList; // no deduplication
+  private Map<String, Integer> columnInfoMap; // used because the server returns deduplicated columns
+  private List<String> columnTypeDeduplicatedList; // deduplicated from columnTypeList
+  private int fetchSize;
+  private boolean emptyResultSet = false;
+
+  private TSQueryNonAlignDataSet tsQueryNonAlignDataSet = null;
+  private byte[][] times; // used for disable align
+  private byte[][] values; // used to cache the current row record value
+
+  private long sessionId;
+  private long queryId;
+  private boolean ignoreTimeStamp = false;
+
+  public IoTDBNonAlignQueryResultSet() {
+    // do nothing
+  }
+
+  // for disable align clause
+  public IoTDBNonAlignQueryResultSet(Statement statement, List<String> columnNameList,
+                                     List<String> columnTypeList, boolean ignoreTimeStamp, TSIService.Iface client,
+                                     String sql, long queryId, long sessionId, TSQueryNonAlignDataSet dataset)
+          throws SQLException {
+    this.statement = statement;
+    this.fetchSize = statement.getFetchSize();
+    this.columnTypeList = columnTypeList;
+
+    times = new byte[columnNameList.size()][Long.BYTES];
+    values = new byte[columnNameList.size()][];
+
+    this.columnInfoList = new ArrayList<>();
+    // deduplicate and map
+    this.columnInfoMap = new HashMap<>();
+    this.columnInfoMap.put(TIMESTAMP_STR, 1);
+    this.columnTypeDeduplicatedList = new ArrayList<>();
+    int index = START_INDEX;
+    for (int i = 0; i < columnNameList.size(); i++) {
+      String name = columnNameList.get(i);
+      columnInfoList.add(TIMESTAMP_STR + name);
+      columnInfoList.add(name);
+      if (!columnInfoMap.containsKey(name)) {
+        columnInfoMap.put(name, index++);
+        columnTypeDeduplicatedList.add(columnTypeList.get(i));
+      }
+    }
+
+    this.ignoreTimeStamp = ignoreTimeStamp;
+    this.client = client;
+    this.sql = sql;
+    this.queryId = queryId;
+    this.tsQueryNonAlignDataSet = dataset;
+    this.sessionId = sessionId;
+  }
+
+  @Override
+  public boolean isWrapperFor(Class<?> iface) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public <T> T unwrap(Class<T> iface) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean absolute(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void afterLast() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void beforeFirst() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void cancelRowUpdates() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void clearWarnings() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void close() throws SQLException {
+    if (isClosed) {
+      return;
+    }
+    if (client != null) {
+      try {
+        TSCloseOperationReq closeReq = new TSCloseOperationReq(sessionId);
+        closeReq.setQueryId(queryId);
+        TSStatus closeResp = client.closeOperation(closeReq);
+        RpcUtils.verifySuccess(closeResp);
+      } catch (IoTDBRPCException e) {
+        throw new SQLException("Error occurs for close opeation in server side becasuse " + e);
+      } catch (TException e) {
+        throw new SQLException(
+                "Error occurs when connecting to server for close operation, becasue: " + e);
+      }
+    }
+    client = null;
+    isClosed = true;
+  }
+
+
+  @Override
+  public void deleteRow() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int findColumn(String columnName) {
+    return columnInfoMap.get(columnName);
+  }
+
+  @Override
+  public boolean first() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Array getArray(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Array getArray(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public InputStream getAsciiStream(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public InputStream getAsciiStream(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(int columnIndex) throws SQLException {
+    return getBigDecimal(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(String columnName) throws SQLException {
+    return new BigDecimal(Objects.requireNonNull(getValueByName(columnName)));
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(int columnIndex, int scale) throws SQLException {
+    MathContext mc = new MathContext(scale);
+    return getBigDecimal(columnIndex).round(mc);
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(String columnName, int scale) throws SQLException {
+    return getBigDecimal(findColumn(columnName), scale);
+  }
+
+  @Override
+  public InputStream getBinaryStream(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public InputStream getBinaryStream(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Blob getBlob(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Blob getBlob(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean getBoolean(int columnIndex) throws SQLException {
+    return getBoolean(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public boolean getBoolean(String columnName) throws SQLException {
+    checkRecord();
+    int index = columnInfoMap.get(columnName) - START_INDEX;
+    if (values[index] != null) {
+      return BytesUtils.bytesToBool(values[index]);
+    }
+    else {
+      throw new SQLException(String.format(VALUE_IS_NULL, columnName));
+    }
+  }
+
+  @Override
+  public byte getByte(int columnIndex) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public byte getByte(String columnName) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public byte[] getBytes(int columnIndex) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public byte[] getBytes(String columnName) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Reader getCharacterStream(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Reader getCharacterStream(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Clob getClob(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Clob getClob(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int getConcurrency() {
+    return ResultSet.CONCUR_READ_ONLY;
+  }
+
+  @Override
+  public String getCursorName() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Date getDate(int columnIndex) throws SQLException {
+    return new Date(getLong(columnIndex));
+  }
+
+  @Override
+  public Date getDate(String columnName) throws SQLException {
+    return getDate(findColumn(columnName));
+  }
+
+  @Override
+  public Date getDate(int arg0, Calendar arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Date getDate(String arg0, Calendar arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public double getDouble(int columnIndex) throws SQLException {
+    return getDouble(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public double getDouble(String columnName) throws SQLException {
+    checkRecord();
+    int index = columnInfoMap.get(columnName) - START_INDEX;
+    if (values[index] != null) {
+      return BytesUtils.bytesToDouble(values[index]);
+    } else {
+      throw new SQLException(String.format(VALUE_IS_NULL, columnName));
+    }
+  }
+
+  @Override
+  public int getFetchDirection() throws SQLException {
+    return ResultSet.FETCH_FORWARD;
+  }
+
+  @Override
+  public void setFetchDirection(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int getFetchSize() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void setFetchSize(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public float getFloat(int columnIndex) throws SQLException {
+    return getFloat(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public float getFloat(String columnName) throws SQLException {
+    checkRecord();
+    int index = columnInfoMap.get(columnName) - START_INDEX;
+    if (values[index] != null) {
+      return BytesUtils.bytesToFloat(values[index]);
+    } else {
+      throw new SQLException(String.format(VALUE_IS_NULL, columnName));
+    }
+  }
+
+  @Override
+  public int getHoldability() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int getInt(int columnIndex) throws SQLException {
+    return getInt(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public int getInt(String columnName) throws SQLException {
+    checkRecord();
+    int index = columnInfoMap.get(columnName) - START_INDEX;
+    if (values[index] != null) {
+      return BytesUtils.bytesToInt(values[index]);
+    } else {
+      throw new SQLException(String.format(VALUE_IS_NULL, columnName));
+    }
+  }
+
+  @Override
+  public long getLong(int columnIndex) throws SQLException {
+    return getLong(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public long getLong(String columnName) throws SQLException {
+    checkRecord();
+    if (columnName.startsWith(TIMESTAMP_STR)) {
+      String column = columnName.substring(TIMESTAMP_STR_LENGTH);
+      int index = columnInfoMap.get(column) - START_INDEX;
+      return BytesUtils.bytesToLong(times[index]);
+    }
+    int index = columnInfoMap.get(columnName) - START_INDEX;
+    if (values[index] != null) {
+      return BytesUtils.bytesToLong(values[index]);
+    } else {
+      throw new SQLException(String.format(VALUE_IS_NULL, columnName));
+    }
+  }
+
+  @Override
+  public ResultSetMetaData getMetaData() {
+    return new IoTDBResultMetadata(columnInfoList, columnTypeList, false);
+  }
+
+  @Override
+  public Reader getNCharacterStream(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Reader getNCharacterStream(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public NClob getNClob(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public NClob getNClob(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public String getNString(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public String getNString(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Object getObject(int columnIndex) throws SQLException {
+    return getObject(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public Object getObject(String columnName) throws SQLException {
+    return getValueByName(columnName);
+  }
+
+  @Override
+  public Object getObject(int arg0, Map<String, Class<?>> arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Object getObject(String arg0, Map<String, Class<?>> arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public <T> T getObject(int arg0, Class<T> arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public <T> T getObject(String arg0, Class<T> arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Ref getRef(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Ref getRef(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int getRow() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public RowId getRowId(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public RowId getRowId(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public SQLXML getSQLXML(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public SQLXML getSQLXML(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public short getShort(int columnIndex) throws SQLException {
+    return getShort(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public short getShort(String columnName) throws SQLException {
+    return Short.parseShort(Objects.requireNonNull(getValueByName(columnName)));
+  }
+
+  @Override
+  public Statement getStatement() {
+    return this.statement;
+  }
+
+  @Override
+  public String getString(int columnIndex) throws SQLException {
+    return getString(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public String getString(String columnName) throws SQLException {
+    return getValueByName(columnName);
+  }
+
+  @Override
+  public Time getTime(int columnIndex) throws SQLException {
+    return new Time(getLong(columnIndex));
+  }
+
+  @Override
+  public Time getTime(String columnName) throws SQLException {
+    return getTime(findColumn(columnName));
+  }
+
+  @Override
+  public Time getTime(int arg0, Calendar arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Time getTime(String arg0, Calendar arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Timestamp getTimestamp(int columnIndex) throws SQLException {
+    return new Timestamp(getLong(columnIndex));
+  }
+
+  @Override
+  public Timestamp getTimestamp(String columnName) throws SQLException {
+    return getTimestamp(findColumn(columnName));
+  }
+
+  @Override
+  public Timestamp getTimestamp(int arg0, Calendar arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Timestamp getTimestamp(String arg0, Calendar arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int getType() {
+    return ResultSet.TYPE_FORWARD_ONLY;
+  }
+
+  @Override
+  public URL getURL(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public URL getURL(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public InputStream getUnicodeStream(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public InputStream getUnicodeStream(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public SQLWarning getWarnings() {
+    return warningChain;
+  }
+
+  @Override
+  public void insertRow() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean isAfterLast() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean isBeforeFirst() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean isClosed() {
+    return isClosed;
+  }
+
+  @Override
+  public boolean isFirst() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean isLast() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean last() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void moveToCurrentRow() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void moveToInsertRow() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean next() throws SQLException {
+    if (hasCachedResults()) {
+      constructNonAlignOneRow();
+      return true;
+    }
+    if (emptyResultSet) {
+      return false;
+    }
+    if (fetchResults()) {
+      constructNonAlignOneRow();
+      return true;
+    }
+    return false;
+  }
+
+
+  /**
+   * @return true means has results
+   */
+  private boolean fetchResults() throws SQLException {
+    TSFetchResultsReq req = new TSFetchResultsReq(sessionId, sql, fetchSize, queryId, false);
+    try {
+      TSFetchResultsResp resp = client.fetchResults(req);
+
+      try {
+        RpcUtils.verifySuccess(resp.getStatus());
+      } catch (IoTDBRPCException e) {
+        throw new IoTDBSQLException(e.getMessage(), resp.getStatus());
+      }
+      if (!resp.hasResultSet) {
+        emptyResultSet = true;
+      } else {
+        tsQueryNonAlignDataSet = resp.getNonAlignQueryDataSet();
+        if (tsQueryNonAlignDataSet == null) {
+          return false;
+        }
+      }
+      return resp.hasResultSet;
+    } catch (TException e) {
+      throw new SQLException(
+              "Cannot fetch result from server, because of network connection: {} ", e);
+    }
+  }
+
+  private boolean hasCachedResults() {
+    return (tsQueryNonAlignDataSet != null && hasTimesRemaining());
+  }
+
+  // check if has times remaining for disable align clause
+  private boolean hasTimesRemaining() {
+    for (ByteBuffer time : tsQueryNonAlignDataSet.timeList) {
+      if (time.hasRemaining()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private void constructNonAlignOneRow() {
+    for (int i = 0; i < tsQueryNonAlignDataSet.timeList.size(); i++) {
+      times[i] = null;
+      values[i] = null;
+      if (tsQueryNonAlignDataSet.timeList.get(i).remaining() >= Long.BYTES) {
+        if (times[i] == null) {
+          times[i] = new byte[Long.BYTES];
+        }
 
 Review comment:
   You set `times[i]` and `values[i]` to null every time, so I can not see the point of `if (times[i] == null)`.
   By the way, I think it is not necessary to copy the data out of the ByteBuffer, you may just get the value from the origin ByteBuffer (except the text type). The following code explains how to do so in a simplified way.
   ```
   private int rowIndex;
   ...
   public long getLong(String columnName) {
       int columnNum = positionOfColumn(columnName);
       ByteBuffer buffer = tsQueryNonAlignDataSet.valueList.get(columnNum);
       if (buffer.limit() < rowIndex * Long.BYTES) {
           // report a null value
       } else {
           buffer.position(rowIndex * Long.BYTES);
           return buffer.getLong();
       }
   }
   
   public void next() {
     // check if the index of out all ranges
     rowIndex ++;
   }
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366280738
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/NonAlignEngineDataSet.java
 ##########
 @@ -0,0 +1,354 @@
+/*
+ * 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.dataset;
+
+import org.apache.iotdb.db.query.pool.QueryTaskPoolManager;
+import org.apache.iotdb.db.query.reader.ManagedSeriesReader;
+import org.apache.iotdb.db.tools.watermark.WatermarkEncoder;
+import org.apache.iotdb.service.rpc.thrift.TSQueryNonAlignDataSet;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicIntegerArray;
+
+public class NonAlignEngineDataSet extends QueryDataSet {
+
+  private static class ReadTask implements Runnable {
+
+    private final ManagedSeriesReader reader;
+    private BlockingQueue<Pair<ByteBuffer, ByteBuffer>> blockingQueue;
+    private WatermarkEncoder encoder;
+    NonAlignEngineDataSet dataSet;
+    private int index;
+
+
+    public ReadTask(ManagedSeriesReader reader,
+                    BlockingQueue<Pair<ByteBuffer, ByteBuffer>> blockingQueue,
+                    WatermarkEncoder encoder, NonAlignEngineDataSet dataSet, int index) {
+      this.reader = reader;
+      this.blockingQueue = blockingQueue;
+      this.encoder = encoder;
+      this.dataSet = dataSet;
+      this.index = index;
+    }
+
+    @Override
+    public void run() {
+      PublicBAOS timeBAOS = new PublicBAOS();
+      PublicBAOS valueBAOS = new PublicBAOS();
+      try {
+        synchronized (reader) {
+          // if the task is submitted, there must be free space in the queue
+          // so here we don't need to check whether the queue has free space
+          // the reader has next batch
+          if ((dataSet.cachedBatchData[index] != null && dataSet.cachedBatchData[index].hasCurrent())
+                  || reader.hasNextBatch()) {
+            BatchData batchData;
+            if (dataSet.cachedBatchData[index] != null && dataSet.cachedBatchData[index].hasCurrent())
+              batchData = dataSet.cachedBatchData[index];
+            else
+              batchData = reader.nextBatch();
+
+            int rowCount = 0;
+            while (rowCount < dataSet.fetchSize) {
+
+              if ((dataSet.limit > 0 && dataSet.alreadyReturnedRowNumArray.get(index) >= dataSet.limit)) {
+                break;
+              }
+
+              if (batchData != null && batchData.hasCurrent()) {
+                if (dataSet.offsetArray.get(index) == 0) {
+                  long time = batchData.currentTime();
+                  ReadWriteIOUtils.write(time, timeBAOS);
+                  TSDataType type = batchData.getDataType();
+                  switch (type) {
+                    case INT32:
+                      int intValue = batchData.getInt();
+                      if (encoder != null && encoder.needEncode(time)) {
+                        intValue = encoder.encodeInt(intValue, time);
+                      }
+                      ReadWriteIOUtils.write(intValue, valueBAOS);
+                      break;
+                    case INT64:
+                      long longValue = batchData.getLong();
+                      if (encoder != null && encoder.needEncode(time)) {
+                        longValue = encoder.encodeLong(longValue, time);
+                      }
+                      ReadWriteIOUtils.write(longValue, valueBAOS);
+                      break;
+                    case FLOAT:
+                      float floatValue = batchData.getFloat();
+                      if (encoder != null && encoder.needEncode(time)) {
+                        floatValue = encoder.encodeFloat(floatValue, time);
+                      }
+                      ReadWriteIOUtils.write(floatValue, valueBAOS);
+                      break;
+                    case DOUBLE:
+                      double doubleValue = batchData.getDouble();
+                      if (encoder != null && encoder.needEncode(time)) {
+                        doubleValue = encoder.encodeDouble(doubleValue, time);
+                      }
+                      ReadWriteIOUtils.write(doubleValue, valueBAOS);
+                      break;
+                    case BOOLEAN:
+                      ReadWriteIOUtils.write(batchData.getBoolean(),
+                              valueBAOS);
+                      break;
+                    case TEXT:
+                      ReadWriteIOUtils
+                              .write(batchData.getBinary(),
+                                      valueBAOS);
+                      break;
+                    default:
+                      throw new UnSupportedDataTypeException(
+                              String.format("Data type %s is not supported.", type));
+                  }
+                }
+                batchData.next();
+              }
+              else {
+                if (reader.hasNextBatch()) {
+                  batchData = reader.nextBatch();
+                  dataSet.cachedBatchData[index] = batchData;
+                  continue;
+                }
+                else
+                  break;
+              }
+              if (dataSet.offsetArray.get(index) == 0) {
+                rowCount++;
+                if (dataSet.limit > 0) {
+                  dataSet.alreadyReturnedRowNumArray.incrementAndGet(index);
+                }
+              } else {
+                dataSet.offsetArray.decrementAndGet(index);
+              }
+            }
+            if (rowCount == 0) {
+              blockingQueue.put(new Pair(null, null));
+              // set the hasRemaining field in reader to false
+              // tell the Consumer not to submit another task for this reader any more
+              reader.setHasRemaining(false);
+              // remove itself from the QueryTaskPoolManager
+              reader.setManagedByQueryManager(false);
+              return;
+            }
+
+            ByteBuffer timeBuffer = ByteBuffer.allocate(timeBAOS.size());
+            timeBuffer.put(timeBAOS.getBuf(), 0, timeBAOS.size());
+            timeBuffer.flip();
+            ByteBuffer valueBuffer = ByteBuffer.allocate(valueBAOS.size());
+            valueBuffer.put(valueBAOS.getBuf(), 0, valueBAOS.size());
+            valueBuffer.flip();
 
 Review comment:
   You may just wrap the byte[] in the BAOS with `ByteBuffer.wrap()` as it does not seem you will reuse it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366266157
 
 

 ##########
 File path: jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBNonAlignQueryResultSet.java
 ##########
 @@ -0,0 +1,1289 @@
+/*
+ * 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.jdbc;
+
+import org.apache.iotdb.rpc.IoTDBRPCException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.service.rpc.thrift.*;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.thrift.TException;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.sql.Date;
+import java.sql.*;
+import java.util.*;
+
+public class IoTDBNonAlignQueryResultSet implements ResultSet {
+
+  private static final String TIMESTAMP_STR = "Time";
+  private static final int TIMESTAMP_STR_LENGTH = 4;
+  private static final int START_INDEX = 2;
+  private static final String VALUE_IS_NULL = "The value got by %s (column name) is NULL.";
+  private static final String EMPTY_STR = "";
+  private Statement statement = null;
+  private String sql;
+  private SQLWarning warningChain = null;
+  private boolean isClosed = false;
+  private TSIService.Iface client = null;
+  private List<String> columnInfoList; // no deduplication
+  private List<String> columnTypeList; // no deduplication
+  private Map<String, Integer> columnInfoMap; // used because the server returns deduplicated columns
+  private List<String> columnTypeDeduplicatedList; // deduplicated from columnTypeList
+  private int fetchSize;
+  private boolean emptyResultSet = false;
+
+  private TSQueryNonAlignDataSet tsQueryNonAlignDataSet = null;
+  private byte[][] times; // used for disable align
+  private byte[][] values; // used to cache the current row record value
+
+  private long sessionId;
+  private long queryId;
+  private boolean ignoreTimeStamp = false;
+
+  public IoTDBNonAlignQueryResultSet() {
+    // do nothing
+  }
+
+  // for disable align clause
+  public IoTDBNonAlignQueryResultSet(Statement statement, List<String> columnNameList,
+                                     List<String> columnTypeList, boolean ignoreTimeStamp, TSIService.Iface client,
+                                     String sql, long queryId, long sessionId, TSQueryNonAlignDataSet dataset)
+          throws SQLException {
+    this.statement = statement;
+    this.fetchSize = statement.getFetchSize();
+    this.columnTypeList = columnTypeList;
+
+    times = new byte[columnNameList.size()][Long.BYTES];
+    values = new byte[columnNameList.size()][];
+
+    this.columnInfoList = new ArrayList<>();
+    // deduplicate and map
+    this.columnInfoMap = new HashMap<>();
+    this.columnInfoMap.put(TIMESTAMP_STR, 1);
+    this.columnTypeDeduplicatedList = new ArrayList<>();
+    int index = START_INDEX;
+    for (int i = 0; i < columnNameList.size(); i++) {
+      String name = columnNameList.get(i);
+      columnInfoList.add(TIMESTAMP_STR + name);
+      columnInfoList.add(name);
+      if (!columnInfoMap.containsKey(name)) {
+        columnInfoMap.put(name, index++);
+        columnTypeDeduplicatedList.add(columnTypeList.get(i));
+      }
+    }
+
+    this.ignoreTimeStamp = ignoreTimeStamp;
+    this.client = client;
+    this.sql = sql;
+    this.queryId = queryId;
+    this.tsQueryNonAlignDataSet = dataset;
+    this.sessionId = sessionId;
+  }
+
+  @Override
+  public boolean isWrapperFor(Class<?> iface) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public <T> T unwrap(Class<T> iface) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean absolute(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void afterLast() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void beforeFirst() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void cancelRowUpdates() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void clearWarnings() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void close() throws SQLException {
+    if (isClosed) {
+      return;
+    }
+    if (client != null) {
+      try {
+        TSCloseOperationReq closeReq = new TSCloseOperationReq(sessionId);
+        closeReq.setQueryId(queryId);
+        TSStatus closeResp = client.closeOperation(closeReq);
+        RpcUtils.verifySuccess(closeResp);
+      } catch (IoTDBRPCException e) {
+        throw new SQLException("Error occurs for close opeation in server side becasuse " + e);
+      } catch (TException e) {
+        throw new SQLException(
+                "Error occurs when connecting to server for close operation, becasue: " + e);
+      }
+    }
+    client = null;
+    isClosed = true;
+  }
+
+
+  @Override
+  public void deleteRow() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int findColumn(String columnName) {
+    return columnInfoMap.get(columnName);
+  }
+
+  @Override
+  public boolean first() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Array getArray(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Array getArray(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public InputStream getAsciiStream(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public InputStream getAsciiStream(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(int columnIndex) throws SQLException {
+    return getBigDecimal(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(String columnName) throws SQLException {
+    return new BigDecimal(Objects.requireNonNull(getValueByName(columnName)));
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(int columnIndex, int scale) throws SQLException {
+    MathContext mc = new MathContext(scale);
+    return getBigDecimal(columnIndex).round(mc);
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(String columnName, int scale) throws SQLException {
+    return getBigDecimal(findColumn(columnName), scale);
+  }
+
+  @Override
+  public InputStream getBinaryStream(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public InputStream getBinaryStream(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Blob getBlob(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Blob getBlob(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean getBoolean(int columnIndex) throws SQLException {
+    return getBoolean(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public boolean getBoolean(String columnName) throws SQLException {
+    checkRecord();
+    int index = columnInfoMap.get(columnName) - START_INDEX;
+    if (values[index] != null) {
+      return BytesUtils.bytesToBool(values[index]);
+    }
+    else {
+      throw new SQLException(String.format(VALUE_IS_NULL, columnName));
+    }
+  }
+
+  @Override
+  public byte getByte(int columnIndex) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public byte getByte(String columnName) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public byte[] getBytes(int columnIndex) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public byte[] getBytes(String columnName) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Reader getCharacterStream(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Reader getCharacterStream(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Clob getClob(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Clob getClob(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int getConcurrency() {
+    return ResultSet.CONCUR_READ_ONLY;
+  }
+
+  @Override
+  public String getCursorName() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Date getDate(int columnIndex) throws SQLException {
+    return new Date(getLong(columnIndex));
+  }
+
+  @Override
+  public Date getDate(String columnName) throws SQLException {
+    return getDate(findColumn(columnName));
+  }
+
+  @Override
+  public Date getDate(int arg0, Calendar arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Date getDate(String arg0, Calendar arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public double getDouble(int columnIndex) throws SQLException {
+    return getDouble(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public double getDouble(String columnName) throws SQLException {
+    checkRecord();
+    int index = columnInfoMap.get(columnName) - START_INDEX;
+    if (values[index] != null) {
+      return BytesUtils.bytesToDouble(values[index]);
+    } else {
+      throw new SQLException(String.format(VALUE_IS_NULL, columnName));
+    }
+  }
+
+  @Override
+  public int getFetchDirection() throws SQLException {
+    return ResultSet.FETCH_FORWARD;
+  }
+
+  @Override
+  public void setFetchDirection(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int getFetchSize() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void setFetchSize(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public float getFloat(int columnIndex) throws SQLException {
+    return getFloat(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public float getFloat(String columnName) throws SQLException {
+    checkRecord();
+    int index = columnInfoMap.get(columnName) - START_INDEX;
+    if (values[index] != null) {
+      return BytesUtils.bytesToFloat(values[index]);
+    } else {
+      throw new SQLException(String.format(VALUE_IS_NULL, columnName));
+    }
+  }
+
+  @Override
+  public int getHoldability() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int getInt(int columnIndex) throws SQLException {
+    return getInt(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public int getInt(String columnName) throws SQLException {
+    checkRecord();
+    int index = columnInfoMap.get(columnName) - START_INDEX;
+    if (values[index] != null) {
+      return BytesUtils.bytesToInt(values[index]);
+    } else {
+      throw new SQLException(String.format(VALUE_IS_NULL, columnName));
+    }
+  }
+
+  @Override
+  public long getLong(int columnIndex) throws SQLException {
+    return getLong(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public long getLong(String columnName) throws SQLException {
+    checkRecord();
+    if (columnName.startsWith(TIMESTAMP_STR)) {
+      String column = columnName.substring(TIMESTAMP_STR_LENGTH);
+      int index = columnInfoMap.get(column) - START_INDEX;
+      return BytesUtils.bytesToLong(times[index]);
+    }
 
 Review comment:
   I have to say, your current way of dealing with null values is not satisfactory, maybe a bitmap will help.
   ![image](https://user-images.githubusercontent.com/23610645/72337333-5b068080-36fd-11ea-99d7-533864091a56.png)
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] sonarcloud[bot] commented on issue #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on issue #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#issuecomment-574479916
 
 
   Kudos, SonarCloud Quality Gate passed!
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) (and [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) to review)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [12 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo.png' alt='No Coverage information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738) No Coverage information  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3.png' alt='0.0%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list)
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366279675
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/NonAlignEngineDataSet.java
 ##########
 @@ -0,0 +1,354 @@
+/*
+ * 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.dataset;
+
+import org.apache.iotdb.db.query.pool.QueryTaskPoolManager;
+import org.apache.iotdb.db.query.reader.ManagedSeriesReader;
+import org.apache.iotdb.db.tools.watermark.WatermarkEncoder;
+import org.apache.iotdb.service.rpc.thrift.TSQueryNonAlignDataSet;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicIntegerArray;
+
+public class NonAlignEngineDataSet extends QueryDataSet {
+
+  private static class ReadTask implements Runnable {
+
+    private final ManagedSeriesReader reader;
+    private BlockingQueue<Pair<ByteBuffer, ByteBuffer>> blockingQueue;
+    private WatermarkEncoder encoder;
+    NonAlignEngineDataSet dataSet;
 
 Review comment:
   This field is not necessary if you make ReadTask a non-static class.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] sonarcloud[bot] removed a comment on issue #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on issue #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#issuecomment-574130028
 
 
   SonarCloud Quality Gate failed.
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug.png' alt='Bug' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C.png' alt='C' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=BUG)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability.png' alt='Vulnerability' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=VULNERABILITY) (and [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot.png' alt='Security Hotspot' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=SECURITY_HOTSPOT) to review)  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell.png' alt='Code Smell' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A.png' alt='A' width='16' height='16' />](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL) [15 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=738&resolved=false&types=CODE_SMELL)
   
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo.png' alt='No Coverage information' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738) No Coverage information  
   [<img src='https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/20plus.png' alt='46.7%' width='16' height='16' />](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list) [46.7% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=738&metric=new_duplicated_lines_density&view=list)
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] JackieTien97 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366325149
 
 

 ##########
 File path: jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBNonAlignQueryResultSet.java
 ##########
 @@ -0,0 +1,1289 @@
+/*
+ * 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.jdbc;
+
+import org.apache.iotdb.rpc.IoTDBRPCException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.service.rpc.thrift.*;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.thrift.TException;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.sql.Date;
+import java.sql.*;
+import java.util.*;
+
+public class IoTDBNonAlignQueryResultSet implements ResultSet {
+
+  private static final String TIMESTAMP_STR = "Time";
+  private static final int TIMESTAMP_STR_LENGTH = 4;
+  private static final int START_INDEX = 2;
+  private static final String VALUE_IS_NULL = "The value got by %s (column name) is NULL.";
+  private static final String EMPTY_STR = "";
+  private Statement statement = null;
+  private String sql;
+  private SQLWarning warningChain = null;
+  private boolean isClosed = false;
+  private TSIService.Iface client = null;
+  private List<String> columnInfoList; // no deduplication
+  private List<String> columnTypeList; // no deduplication
+  private Map<String, Integer> columnInfoMap; // used because the server returns deduplicated columns
+  private List<String> columnTypeDeduplicatedList; // deduplicated from columnTypeList
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] JackieTien97 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366322162
 
 

 ##########
 File path: jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBNonAlignQueryResultSet.java
 ##########
 @@ -0,0 +1,1289 @@
+/*
+ * 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.jdbc;
+
+import org.apache.iotdb.rpc.IoTDBRPCException;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.service.rpc.thrift.*;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.thrift.TException;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.sql.Date;
+import java.sql.*;
+import java.util.*;
+
+public class IoTDBNonAlignQueryResultSet implements ResultSet {
+
+  private static final String TIMESTAMP_STR = "Time";
+  private static final int TIMESTAMP_STR_LENGTH = 4;
+  private static final int START_INDEX = 2;
+  private static final String VALUE_IS_NULL = "The value got by %s (column name) is NULL.";
+  private static final String EMPTY_STR = "";
+  private Statement statement = null;
+  private String sql;
+  private SQLWarning warningChain = null;
+  private boolean isClosed = false;
+  private TSIService.Iface client = null;
+  private List<String> columnInfoList; // no deduplication
+  private List<String> columnTypeList; // no deduplication
+  private Map<String, Integer> columnInfoMap; // used because the server returns deduplicated columns
+  private List<String> columnTypeDeduplicatedList; // deduplicated from columnTypeList
+  private int fetchSize;
+  private boolean emptyResultSet = false;
+
+  private TSQueryNonAlignDataSet tsQueryNonAlignDataSet = null;
+  private byte[][] times; // used for disable align
+  private byte[][] values; // used to cache the current row record value
+
+  private long sessionId;
+  private long queryId;
+  private boolean ignoreTimeStamp = false;
+
+  public IoTDBNonAlignQueryResultSet() {
+    // do nothing
+  }
+
+  // for disable align clause
+  public IoTDBNonAlignQueryResultSet(Statement statement, List<String> columnNameList,
+                                     List<String> columnTypeList, boolean ignoreTimeStamp, TSIService.Iface client,
+                                     String sql, long queryId, long sessionId, TSQueryNonAlignDataSet dataset)
+          throws SQLException {
+    this.statement = statement;
+    this.fetchSize = statement.getFetchSize();
+    this.columnTypeList = columnTypeList;
+
+    times = new byte[columnNameList.size()][Long.BYTES];
+    values = new byte[columnNameList.size()][];
+
+    this.columnInfoList = new ArrayList<>();
+    // deduplicate and map
+    this.columnInfoMap = new HashMap<>();
+    this.columnInfoMap.put(TIMESTAMP_STR, 1);
+    this.columnTypeDeduplicatedList = new ArrayList<>();
+    int index = START_INDEX;
+    for (int i = 0; i < columnNameList.size(); i++) {
+      String name = columnNameList.get(i);
+      columnInfoList.add(TIMESTAMP_STR + name);
+      columnInfoList.add(name);
+      if (!columnInfoMap.containsKey(name)) {
+        columnInfoMap.put(name, index++);
+        columnTypeDeduplicatedList.add(columnTypeList.get(i));
+      }
+    }
+
+    this.ignoreTimeStamp = ignoreTimeStamp;
+    this.client = client;
+    this.sql = sql;
+    this.queryId = queryId;
+    this.tsQueryNonAlignDataSet = dataset;
+    this.sessionId = sessionId;
+  }
+
+  @Override
+  public boolean isWrapperFor(Class<?> iface) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public <T> T unwrap(Class<T> iface) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean absolute(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void afterLast() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void beforeFirst() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void cancelRowUpdates() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void clearWarnings() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void close() throws SQLException {
+    if (isClosed) {
+      return;
+    }
+    if (client != null) {
+      try {
+        TSCloseOperationReq closeReq = new TSCloseOperationReq(sessionId);
+        closeReq.setQueryId(queryId);
+        TSStatus closeResp = client.closeOperation(closeReq);
+        RpcUtils.verifySuccess(closeResp);
+      } catch (IoTDBRPCException e) {
+        throw new SQLException("Error occurs for close opeation in server side becasuse " + e);
+      } catch (TException e) {
+        throw new SQLException(
+                "Error occurs when connecting to server for close operation, becasue: " + e);
+      }
+    }
+    client = null;
+    isClosed = true;
+  }
+
+
+  @Override
+  public void deleteRow() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int findColumn(String columnName) {
+    return columnInfoMap.get(columnName);
+  }
+
+  @Override
+  public boolean first() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Array getArray(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Array getArray(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public InputStream getAsciiStream(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public InputStream getAsciiStream(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(int columnIndex) throws SQLException {
+    return getBigDecimal(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(String columnName) throws SQLException {
+    return new BigDecimal(Objects.requireNonNull(getValueByName(columnName)));
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(int columnIndex, int scale) throws SQLException {
+    MathContext mc = new MathContext(scale);
+    return getBigDecimal(columnIndex).round(mc);
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(String columnName, int scale) throws SQLException {
+    return getBigDecimal(findColumn(columnName), scale);
+  }
+
+  @Override
+  public InputStream getBinaryStream(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public InputStream getBinaryStream(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Blob getBlob(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Blob getBlob(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean getBoolean(int columnIndex) throws SQLException {
+    return getBoolean(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public boolean getBoolean(String columnName) throws SQLException {
+    checkRecord();
+    int index = columnInfoMap.get(columnName) - START_INDEX;
+    if (values[index] != null) {
+      return BytesUtils.bytesToBool(values[index]);
+    }
+    else {
+      throw new SQLException(String.format(VALUE_IS_NULL, columnName));
+    }
+  }
+
+  @Override
+  public byte getByte(int columnIndex) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public byte getByte(String columnName) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public byte[] getBytes(int columnIndex) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public byte[] getBytes(String columnName) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Reader getCharacterStream(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Reader getCharacterStream(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Clob getClob(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Clob getClob(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int getConcurrency() {
+    return ResultSet.CONCUR_READ_ONLY;
+  }
+
+  @Override
+  public String getCursorName() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Date getDate(int columnIndex) throws SQLException {
+    return new Date(getLong(columnIndex));
+  }
+
+  @Override
+  public Date getDate(String columnName) throws SQLException {
+    return getDate(findColumn(columnName));
+  }
+
+  @Override
+  public Date getDate(int arg0, Calendar arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Date getDate(String arg0, Calendar arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public double getDouble(int columnIndex) throws SQLException {
+    return getDouble(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public double getDouble(String columnName) throws SQLException {
+    checkRecord();
+    int index = columnInfoMap.get(columnName) - START_INDEX;
+    if (values[index] != null) {
+      return BytesUtils.bytesToDouble(values[index]);
+    } else {
+      throw new SQLException(String.format(VALUE_IS_NULL, columnName));
+    }
+  }
+
+  @Override
+  public int getFetchDirection() throws SQLException {
+    return ResultSet.FETCH_FORWARD;
+  }
+
+  @Override
+  public void setFetchDirection(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int getFetchSize() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void setFetchSize(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public float getFloat(int columnIndex) throws SQLException {
+    return getFloat(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public float getFloat(String columnName) throws SQLException {
+    checkRecord();
+    int index = columnInfoMap.get(columnName) - START_INDEX;
+    if (values[index] != null) {
+      return BytesUtils.bytesToFloat(values[index]);
+    } else {
+      throw new SQLException(String.format(VALUE_IS_NULL, columnName));
+    }
+  }
+
+  @Override
+  public int getHoldability() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int getInt(int columnIndex) throws SQLException {
+    return getInt(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public int getInt(String columnName) throws SQLException {
+    checkRecord();
+    int index = columnInfoMap.get(columnName) - START_INDEX;
+    if (values[index] != null) {
+      return BytesUtils.bytesToInt(values[index]);
+    } else {
+      throw new SQLException(String.format(VALUE_IS_NULL, columnName));
+    }
+  }
+
+  @Override
+  public long getLong(int columnIndex) throws SQLException {
+    return getLong(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public long getLong(String columnName) throws SQLException {
+    checkRecord();
+    if (columnName.startsWith(TIMESTAMP_STR)) {
+      String column = columnName.substring(TIMESTAMP_STR_LENGTH);
+      int index = columnInfoMap.get(column) - START_INDEX;
+      return BytesUtils.bytesToLong(times[index]);
+    }
+    int index = columnInfoMap.get(columnName) - START_INDEX;
+    if (values[index] != null) {
+      return BytesUtils.bytesToLong(values[index]);
+    } else {
+      throw new SQLException(String.format(VALUE_IS_NULL, columnName));
+    }
+  }
+
+  @Override
+  public ResultSetMetaData getMetaData() {
+    return new IoTDBResultMetadata(columnInfoList, columnTypeList, false);
+  }
+
+  @Override
+  public Reader getNCharacterStream(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Reader getNCharacterStream(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public NClob getNClob(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public NClob getNClob(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public String getNString(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public String getNString(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Object getObject(int columnIndex) throws SQLException {
+    return getObject(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public Object getObject(String columnName) throws SQLException {
+    return getValueByName(columnName);
+  }
+
+  @Override
+  public Object getObject(int arg0, Map<String, Class<?>> arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Object getObject(String arg0, Map<String, Class<?>> arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public <T> T getObject(int arg0, Class<T> arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public <T> T getObject(String arg0, Class<T> arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Ref getRef(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Ref getRef(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int getRow() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public RowId getRowId(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public RowId getRowId(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public SQLXML getSQLXML(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public SQLXML getSQLXML(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public short getShort(int columnIndex) throws SQLException {
+    return getShort(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public short getShort(String columnName) throws SQLException {
+    return Short.parseShort(Objects.requireNonNull(getValueByName(columnName)));
+  }
+
+  @Override
+  public Statement getStatement() {
+    return this.statement;
+  }
+
+  @Override
+  public String getString(int columnIndex) throws SQLException {
+    return getString(findColumnNameByIndex(columnIndex));
+  }
+
+  @Override
+  public String getString(String columnName) throws SQLException {
+    return getValueByName(columnName);
+  }
+
+  @Override
+  public Time getTime(int columnIndex) throws SQLException {
+    return new Time(getLong(columnIndex));
+  }
+
+  @Override
+  public Time getTime(String columnName) throws SQLException {
+    return getTime(findColumn(columnName));
+  }
+
+  @Override
+  public Time getTime(int arg0, Calendar arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Time getTime(String arg0, Calendar arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Timestamp getTimestamp(int columnIndex) throws SQLException {
+    return new Timestamp(getLong(columnIndex));
+  }
+
+  @Override
+  public Timestamp getTimestamp(String columnName) throws SQLException {
+    return getTimestamp(findColumn(columnName));
+  }
+
+  @Override
+  public Timestamp getTimestamp(int arg0, Calendar arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public Timestamp getTimestamp(String arg0, Calendar arg1) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public int getType() {
+    return ResultSet.TYPE_FORWARD_ONLY;
+  }
+
+  @Override
+  public URL getURL(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public URL getURL(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public InputStream getUnicodeStream(int arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public InputStream getUnicodeStream(String arg0) throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public SQLWarning getWarnings() {
+    return warningChain;
+  }
+
+  @Override
+  public void insertRow() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean isAfterLast() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean isBeforeFirst() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean isClosed() {
+    return isClosed;
+  }
+
+  @Override
+  public boolean isFirst() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean isLast() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean last() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void moveToCurrentRow() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void moveToInsertRow() throws SQLException {
+    throw new SQLException(Constant.METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean next() throws SQLException {
+    if (hasCachedResults()) {
+      constructNonAlignOneRow();
+      return true;
+    }
+    if (emptyResultSet) {
+      return false;
+    }
+    if (fetchResults()) {
+      constructNonAlignOneRow();
+      return true;
+    }
+    return false;
+  }
+
+
+  /**
+   * @return true means has results
+   */
+  private boolean fetchResults() throws SQLException {
+    TSFetchResultsReq req = new TSFetchResultsReq(sessionId, sql, fetchSize, queryId, false);
+    try {
+      TSFetchResultsResp resp = client.fetchResults(req);
+
+      try {
+        RpcUtils.verifySuccess(resp.getStatus());
+      } catch (IoTDBRPCException e) {
+        throw new IoTDBSQLException(e.getMessage(), resp.getStatus());
+      }
+      if (!resp.hasResultSet) {
+        emptyResultSet = true;
+      } else {
+        tsQueryNonAlignDataSet = resp.getNonAlignQueryDataSet();
+        if (tsQueryNonAlignDataSet == null) {
+          return false;
+        }
+      }
+      return resp.hasResultSet;
+    } catch (TException e) {
+      throw new SQLException(
+              "Cannot fetch result from server, because of network connection: {} ", e);
+    }
+  }
+
+  private boolean hasCachedResults() {
+    return (tsQueryNonAlignDataSet != null && hasTimesRemaining());
+  }
+
+  // check if has times remaining for disable align clause
+  private boolean hasTimesRemaining() {
+    for (ByteBuffer time : tsQueryNonAlignDataSet.timeList) {
+      if (time.hasRemaining()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private void constructNonAlignOneRow() {
+    for (int i = 0; i < tsQueryNonAlignDataSet.timeList.size(); i++) {
+      times[i] = null;
+      values[i] = null;
+      if (tsQueryNonAlignDataSet.timeList.get(i).remaining() >= Long.BYTES) {
+        if (times[i] == null) {
+          times[i] = new byte[Long.BYTES];
+        }
 
 Review comment:
   This just imitates the code in IoTDBQueryResultSet.
   Maybe I will change to the way you said next time

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #738: [IOTDB-396] New query clause: disable align
URL: https://github.com/apache/incubator-iotdb/pull/738#discussion_r366295291
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
 ##########
 @@ -802,23 +781,44 @@ public TSFetchResultsResp fetchResults(TSFetchResultsReq req) {
 
       if (!queryId2DataSet.containsKey(req.queryId)) {
         return getTSFetchResultsResp(
-            getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, "Has not executed query"));
+                getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, "Has not executed query"));
       }
 
       QueryDataSet queryDataSet = queryId2DataSet.get(req.queryId);
-      TSQueryDataSet result = fillRpcReturnData(req.fetchSize, queryDataSet,
-          sessionIdUsernameMap.get(req.sessionId));
-
-      boolean hasResultSet = result.bufferForTime().limit() != 0;
-      if (!hasResultSet) {
-        queryId2DataSet.remove(req.queryId);
+      if (req.isAlign) {
+        TSQueryDataSet result = fillRpcReturnData(req.fetchSize, queryDataSet,
+                sessionIdUsernameMap.get(req.sessionId));
+        boolean hasResultSet = result.bufferForTime().limit() != 0;
+        if (!hasResultSet) {
+          queryId2DataSet.remove(req.queryId);
 
 Review comment:
   It may be better to also end this query in case anyone forgets to close the resultset.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services