You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2021/03/26 02:20:13 UTC

[GitHub] [incubator-pinot] Jackie-Jiang opened a new pull request #6719: [WIP] Add TIMESTAMP and BOOLEAN data type support

Jackie-Jiang opened a new pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719


   ## Description
   Will add some tests and link a design doc soon
   
   ## Release Notes
   To be added
   
   ## Documentation
   To be added to https://docs.pinot.apache.org/basics/components/schema#data-types
   


-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r618671632



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java
##########
@@ -737,11 +732,11 @@ public static void removeColumnMetadataInfo(PropertiesConfiguration properties,
    * @return raw index creator
    * @throws IOException
    */
-  public static ForwardIndexCreator getRawIndexCreatorForColumn(File file,
-      ChunkCompressionType compressionType, String column, DataType dataType, int totalDocs,
-      int lengthOfLongestEntry, boolean deriveNumDocsPerChunk, int writerVersion)
+  public static ForwardIndexCreator getRawIndexCreatorForColumn(File file, ChunkCompressionType compressionType,
+      String column, DataType dataType, int totalDocs, int lengthOfLongestEntry, boolean deriveNumDocsPerChunk,
+      int writerVersion)
       throws IOException {
-    switch (dataType) {
+    switch (dataType.getStoredType()) {

Review comment:
       Added some comments in the javadoc of `DataType.getStoredType()`




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620648078



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/TimestampQueriesTest.java
##########
@@ -0,0 +1,229 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+
+
+/**
+ * Queries test for TIMESTAMP data type.
+ */
+public class TimestampQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "TimestampQueriesTest");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+  private static final long BASE_TIMESTAMP = Timestamp.valueOf("2021-01-01 00:00:00").getTime();
+
+  private static final int NUM_RECORDS = 1000;
+
+  private static final String TIMESTAMP_COLUMN = "timestampColumn";
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(TIMESTAMP_COLUMN, DataType.TIMESTAMP).build();
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  @Override
+  protected String getFilter() {
+    return "";
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+
+    List<GenericRow> records = new ArrayList<>(NUM_RECORDS);
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      GenericRow record = new GenericRow();
+      long timestamp = BASE_TIMESTAMP + i;
+      // Insert data in 3 different formats

Review comment:
       Can we try couple of values with timezone as well ?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620631641



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/BooleanQueriesTest.java
##########
@@ -0,0 +1,249 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+
+
+/**
+ * Queries test for BOOLEAN data type.
+ */
+public class BooleanQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "BooleanQueriesTest");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final int NUM_RECORDS = 1000;
+
+  private static final String BOOLEAN_COLUMN = "booleanColumn";
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(BOOLEAN_COLUMN, DataType.BOOLEAN).build();
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  @Override
+  protected String getFilter() {
+    return "";
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+
+    List<GenericRow> records = new ArrayList<>(NUM_RECORDS);
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      GenericRow record = new GenericRow();
+      // Insert data in 6 different formats
+      switch (i % 6) {
+        case 0:
+          record.putValue(BOOLEAN_COLUMN, false);
+          break;
+        case 1:
+          record.putValue(BOOLEAN_COLUMN, 1);
+          break;
+        case 2:
+          record.putValue(BOOLEAN_COLUMN, 0L);

Review comment:
       (nit) might be helpful to add few notes on semantics of BOOLEAN in FieldSpec class just like you have handled multiple different formats here.
   
   Is it correct to say that any signed numerical value > 0 is interpreted as true and everything else as false ?
   
   Also, for source string values, will record reader (or some code in segment generator) throw exception if the source string value for a BOOLEAN data type column is anything apart from "true" or "false" ?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r614491082



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -375,6 +530,16 @@ public double toDouble(Object value) {
       throw new UnsupportedOperationException("Cannot convert value from BYTES to DOUBLE");
     }
 
+    @Override
+    public boolean toBoolean(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from BYTES to BOOLEAN");

Review comment:
       I wouldn't worry too much about that. People are not supposed to use `BYTES` as `BOOLEAN`, and throwing exception might be a safer option to avoid unexpected behavior.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] amrishlal commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
amrishlal commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r612097435



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -775,6 +969,18 @@ public static PinotDataType getPinotDataTypeForIngestion(FieldSpec fieldSpec) {
         return fieldSpec.isSingleValueField() ? PinotDataType.FLOAT : PinotDataType.FLOAT_ARRAY;
       case DOUBLE:
         return fieldSpec.isSingleValueField() ? PinotDataType.DOUBLE : PinotDataType.DOUBLE_ARRAY;
+      case BOOLEAN:
+        if (fieldSpec.isSingleValueField()) {
+          return PinotDataType.BOOLEAN;
+        } else {
+          throw new IllegalStateException("There is no multi-value type for BOOLEAN");

Review comment:
       Any specific reason why no multi-value support for BOOLEAN? Boolean is a primitive type just like Numerical types?

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -775,6 +969,18 @@ public static PinotDataType getPinotDataTypeForIngestion(FieldSpec fieldSpec) {
         return fieldSpec.isSingleValueField() ? PinotDataType.FLOAT : PinotDataType.FLOAT_ARRAY;
       case DOUBLE:
         return fieldSpec.isSingleValueField() ? PinotDataType.DOUBLE : PinotDataType.DOUBLE_ARRAY;
+      case BOOLEAN:
+        if (fieldSpec.isSingleValueField()) {
+          return PinotDataType.BOOLEAN;
+        } else {
+          throw new IllegalStateException("There is no multi-value type for BOOLEAN");
+        }
+      case TIMESTAMP:
+        if (fieldSpec.isSingleValueField()) {
+          return PinotDataType.TIMESTAMP;
+        } else {
+          throw new IllegalStateException("There is no multi-value type for TIMESTAMP");

Review comment:
       Same with Timestamp, would like to see multi-valued support here as well unless there is a very specific reason not to do so?

##########
File path: pinot-common/src/test/java/org/apache/pinot/common/data/DateTimeFormatSpecTest.java
##########
@@ -82,6 +86,8 @@ public void testFromMillisToFormat(String format, long timeMs, String expectedFo
     entries.add(new Object[]{"1:MILLISECONDS:EPOCH", 1498892400000L, "1498892400000"});
     entries.add(new Object[]{"1:HOURS:EPOCH", 0L, "0"});
     entries.add(new Object[]{"5:MINUTES:EPOCH", 1498892400000L, "4996308"});
+    entries.add(new Object[]{"1:MILLISECONDS:TIMESTAMP", Timestamp

Review comment:
       Would be good to see a test case for `1:NANOSECONDS:TIMESTAMP` specially since Pinot supports `NANOSECONDS` and `java.sql.Timestamp` seems to require a separate call to `java.sql.Timestamp.setNanos(...)` to set nanoseconds in timestamp.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620744131



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/TimestampQueriesTest.java
##########
@@ -0,0 +1,229 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.sql.Timestamp;

Review comment:
       Here we should stick with SQL timestamp format. The `java.sql.Timestamp` is used for ser/de only. Internally we can convert it to `java.time` classes for calculation.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] amrishlal commented on pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
amrishlal commented on pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#issuecomment-818402617


   This is certainly good work. Both BOOLEAN and TIMESTAMP would be very useful. I haven't looked at all the files yet, but will look again later. Not sure if I missed them, but would like to see more SQL level test cases for these two data types.


-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r614546866



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -775,6 +969,18 @@ public static PinotDataType getPinotDataTypeForIngestion(FieldSpec fieldSpec) {
         return fieldSpec.isSingleValueField() ? PinotDataType.FLOAT : PinotDataType.FLOAT_ARRAY;
       case DOUBLE:
         return fieldSpec.isSingleValueField() ? PinotDataType.DOUBLE : PinotDataType.DOUBLE_ARRAY;
+      case BOOLEAN:
+        if (fieldSpec.isSingleValueField()) {
+          return PinotDataType.BOOLEAN;
+        } else {
+          throw new IllegalStateException("There is no multi-value type for BOOLEAN");
+        }
+      case TIMESTAMP:
+        if (fieldSpec.isSingleValueField()) {
+          return PinotDataType.TIMESTAMP;
+        } else {
+          throw new IllegalStateException("There is no multi-value type for TIMESTAMP");

Review comment:
       `TIMESTAMP` is mostly used as the time field, which does not allow MV. If required, we can add the support later. Added a todo here.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] amrishlal commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
amrishlal commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r612097355



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -315,6 +408,58 @@ public Double convert(Object value, PinotDataType sourceType) {
     }
   },
 
+  TIMESTAMP {
+    @Override
+    public int toInt(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from TIMESTAMP to INTEGER");
+    }
+
+    @Override
+    public long toLong(Object value) {
+      return ((Timestamp) value).getTime();
+    }
+
+    @Override
+    public float toFloat(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from TIMESTAMP to FLOAT");
+    }
+
+    @Override
+    public double toDouble(Object value) {
+      return ((Timestamp) value).getTime();
+    }
+
+    @Override
+    public boolean toBoolean(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from TIMESTAMP to BOOLEAN");

Review comment:
       I would have done zero as `false` and all other timestamp values as `true`. This convention generalizes to work with everything and type conversion to `BOOLEAN` becomes simple :-)




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r614546131



##########
File path: pinot-common/src/test/java/org/apache/pinot/common/data/DateTimeFormatSpecTest.java
##########
@@ -82,6 +86,8 @@ public void testFromMillisToFormat(String format, long timeMs, String expectedFo
     entries.add(new Object[]{"1:MILLISECONDS:EPOCH", 1498892400000L, "1498892400000"});
     entries.add(new Object[]{"1:HOURS:EPOCH", 0L, "0"});
     entries.add(new Object[]{"5:MINUTES:EPOCH", 1498892400000L, "4996308"});
+    entries.add(new Object[]{"1:MILLISECONDS:TIMESTAMP", Timestamp

Review comment:
       SQL Timestamp always have values in `yyyy-MM-dd HH:mm:ss.SSS` format, and it doesn't allow `nanoseconds` granularity. The first 2 values are actually ignored (similar to date time format which only reads the 4th argument as the format). We should consider changing the order for these arguments for clarity, but that is out of the scope of this PR.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620655228



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/scalar/DateTimeFunctions.java
##########
@@ -228,6 +229,22 @@ public static long fromEpochDaysBucket(long days, long bucket) {
     return TimeUnit.DAYS.toMillis(days * bucket);
   }
 
+  /**
+   * Converts epoch millis to Timestamp
+   */
+  @ScalarFunction
+  public static Timestamp toTimestamp(long millis) {
+    return new Timestamp(millis);
+  }
+
+  /**
+   * Converts Timestamp to epoch millis
+   */
+  @ScalarFunction
+  public static long fromTimestamp(Timestamp timestamp) {

Review comment:
       (nit) consider naming it `fromTimestampToMillisSinceEpoch` or something like that ?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620644657



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/TimestampQueriesTest.java
##########
@@ -0,0 +1,229 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+
+
+/**
+ * Queries test for TIMESTAMP data type.
+ */
+public class TimestampQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "TimestampQueriesTest");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+  private static final long BASE_TIMESTAMP = Timestamp.valueOf("2021-01-01 00:00:00").getTime();

Review comment:
       Adding a few notes on semantics of TIMESTAMP in FieldSpec.DataType will be helpful.
   
   Since the stored type is long, we can store any hours/mins/seconds/millis/micros/nanos since epoch  ?
   
   Also, how do we handle timezone here ?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r618671737



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/DataSchema.java
##########
@@ -215,7 +236,34 @@ public int hashCode() {
   }
 
   public enum ColumnDataType {
-    INT, LONG, FLOAT, DOUBLE, STRING, BYTES, OBJECT, INT_ARRAY, LONG_ARRAY, FLOAT_ARRAY, DOUBLE_ARRAY, STRING_ARRAY;
+    INT,
+    LONG,
+    FLOAT,
+    DOUBLE,
+    BOOLEAN /* Stored as INT */,

Review comment:
       Curious, do we plan other int types in the future? e.g. tinyint, short, int etc.

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -60,6 +63,16 @@ public double toDouble(Object value) {
       return ((Boolean) value) ? 1d : 0d;
     }
 
+    @Override
+    public boolean toBoolean(Object value) {

Review comment:
       shall this return type be `Boolean` to allow null value in future?

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -315,6 +408,58 @@ public Double convert(Object value, PinotDataType sourceType) {
     }
   },
 
+  TIMESTAMP {
+    @Override
+    public int toInt(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from TIMESTAMP to INTEGER");
+    }
+
+    @Override
+    public long toLong(Object value) {
+      return ((Timestamp) value).getTime();
+    }
+
+    @Override
+    public float toFloat(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from TIMESTAMP to FLOAT");
+    }
+
+    @Override
+    public double toDouble(Object value) {
+      return ((Timestamp) value).getTime();
+    }
+
+    @Override
+    public boolean toBoolean(Object value) {
+      return ((Timestamp) value).getTime() != 0;

Review comment:
       whats the meaning of 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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620672041



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/TimestampQueriesTest.java
##########
@@ -0,0 +1,229 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.sql.Timestamp;

Review comment:
       `Both Joda-Time and the original Java date-time classes are now outmoded by the java.time classes built into Java 8, Java 9, and later.` from the thread




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r618655995



##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/utils/TimestampUtils.java
##########
@@ -0,0 +1,65 @@
+/**
+ * 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.pinot.spi.utils;
+
+import java.sql.Timestamp;
+
+
+public class TimestampUtils {
+
+  /**
+   * Parses the given timestamp string into {@link Timestamp}.
+   * <p>Two formats of timestamp are supported:
+   * <ul>
+   *   <li>'yyyy-mm-dd hh:mm:ss[.fffffffff]'</li>
+   *   <li>Millis since epoch</li>
+   * </ul>
+   */
+  public static Timestamp toTimestamp(String timestampString) {
+    try {
+      return Timestamp.valueOf(timestampString);
+    } catch (Exception e) {
+      try {
+        return new Timestamp(Long.parseLong(timestampString));
+      } catch (Exception e1) {
+        throw new IllegalArgumentException(String.format("Invalid timestamp: '%s'", timestampString));

Review comment:
       The only exception we can get here is `NumberFormatException` from `Long.parseLong()`, so IMO logging the invalid timestamp should be enough for debugging purpose.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620627441



##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/data/FieldSpec.java
##########
@@ -352,13 +368,41 @@ public int hashCode() {
   public enum DataType {
     // LIST is for complex lists which is different from multi-value column of primitives
     // STRUCT, MAP and LIST are composable to form a COMPLEX field
-    INT, LONG, FLOAT, DOUBLE, BOOLEAN/* Stored as STRING */, STRING, BYTES, STRUCT, MAP, LIST;
+    INT,
+    LONG,
+    FLOAT,
+    DOUBLE,
+    BOOLEAN /* Stored as INT */,
+    TIMESTAMP /* Stored as LONG */,
+    STRING,
+    BYTES,
+    STRUCT,
+    MAP,
+    LIST;
 
     /**
      * Returns the data type stored in Pinot.
+     * <p>Pinot internally stores data (physical) in INT, LONG, FLOAT, DOUBLE, STRING, BYTES type, other data types
+     * (logical) will be stored as one of these types.
+     * <p>Stored type should be used when reading the physical stored values from Dictionary, Forward Index etc.
      */
     public DataType getStoredType() {
-      return this == BOOLEAN ? STRING : this;
+      switch (this) {
+        case BOOLEAN:
+          return INT;

Review comment:
       Not necessarily in this PR, but would it make sense to store this is a bit / byte as opposed to 32 bits in future ? If there is a plan to do so, may be we can write a property to segment metadata (BOOLEAN_STORED_TYPE= INT) to support evolution




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r614546730



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -775,6 +969,18 @@ public static PinotDataType getPinotDataTypeForIngestion(FieldSpec fieldSpec) {
         return fieldSpec.isSingleValueField() ? PinotDataType.FLOAT : PinotDataType.FLOAT_ARRAY;
       case DOUBLE:
         return fieldSpec.isSingleValueField() ? PinotDataType.DOUBLE : PinotDataType.DOUBLE_ARRAY;
+      case BOOLEAN:
+        if (fieldSpec.isSingleValueField()) {
+          return PinotDataType.BOOLEAN;
+        } else {
+          throw new IllegalStateException("There is no multi-value type for BOOLEAN");

Review comment:
       I feel it should be very rare to have MV boolean field (there are only 2 possible values, and the dimension should be either `true` or `false`). If required, we can add the support later. Added a todo here.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620744508



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/TimestampQueriesTest.java
##########
@@ -0,0 +1,229 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+
+
+/**
+ * Queries test for TIMESTAMP data type.
+ */
+public class TimestampQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "TimestampQueriesTest");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+  private static final long BASE_TIMESTAMP = Timestamp.valueOf("2021-01-01 00:00:00").getTime();
+
+  private static final int NUM_RECORDS = 1000;
+
+  private static final String TIMESTAMP_COLUMN = "timestampColumn";
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(TIMESTAMP_COLUMN, DataType.TIMESTAMP).build();
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  @Override
+  protected String getFilter() {
+    return "";
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+
+    List<GenericRow> records = new ArrayList<>(NUM_RECORDS);
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      GenericRow record = new GenericRow();
+      long timestamp = BASE_TIMESTAMP + i;
+      // Insert data in 3 different formats

Review comment:
       Timezone is not stored in TIMESTAMP type




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620727402



##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/data/FieldSpec.java
##########
@@ -352,13 +368,41 @@ public int hashCode() {
   public enum DataType {
     // LIST is for complex lists which is different from multi-value column of primitives
     // STRUCT, MAP and LIST are composable to form a COMPLEX field
-    INT, LONG, FLOAT, DOUBLE, BOOLEAN/* Stored as STRING */, STRING, BYTES, STRUCT, MAP, LIST;
+    INT,
+    LONG,
+    FLOAT,
+    DOUBLE,
+    BOOLEAN /* Stored as INT */,
+    TIMESTAMP /* Stored as LONG */,
+    STRING,
+    BYTES,
+    STRUCT,
+    MAP,
+    LIST;
 
     /**
      * Returns the data type stored in Pinot.
+     * <p>Pinot internally stores data (physical) in INT, LONG, FLOAT, DOUBLE, STRING, BYTES type, other data types
+     * (logical) will be stored as one of these types.
+     * <p>Stored type should be used when reading the physical stored values from Dictionary, Forward Index etc.
      */
     public DataType getStoredType() {
-      return this == BOOLEAN ? STRING : this;
+      switch (this) {
+        case BOOLEAN:
+          return INT;

Review comment:
       I don't see it very necessary to support bit/byte/short natively. With dictionary encoding, it should perform almost identical to `INT`. If we do want to add them in the future, we can add the metadata then, and make it default to `INT` for backward-compatibility.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620655228



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/scalar/DateTimeFunctions.java
##########
@@ -228,6 +229,22 @@ public static long fromEpochDaysBucket(long days, long bucket) {
     return TimeUnit.DAYS.toMillis(days * bucket);
   }
 
+  /**
+   * Converts epoch millis to Timestamp
+   */
+  @ScalarFunction
+  public static Timestamp toTimestamp(long millis) {
+    return new Timestamp(millis);
+  }
+
+  /**
+   * Converts Timestamp to epoch millis
+   */
+  @ScalarFunction
+  public static long fromTimestamp(Timestamp timestamp) {

Review comment:
       (nit) consider naming it fromTimestampToMillisSinceEpoch or something like that ?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620631641



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/BooleanQueriesTest.java
##########
@@ -0,0 +1,249 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+
+
+/**
+ * Queries test for BOOLEAN data type.
+ */
+public class BooleanQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "BooleanQueriesTest");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final int NUM_RECORDS = 1000;
+
+  private static final String BOOLEAN_COLUMN = "booleanColumn";
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(BOOLEAN_COLUMN, DataType.BOOLEAN).build();
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  @Override
+  protected String getFilter() {
+    return "";
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+
+    List<GenericRow> records = new ArrayList<>(NUM_RECORDS);
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      GenericRow record = new GenericRow();
+      // Insert data in 6 different formats
+      switch (i % 6) {
+        case 0:
+          record.putValue(BOOLEAN_COLUMN, false);
+          break;
+        case 1:
+          record.putValue(BOOLEAN_COLUMN, 1);
+          break;
+        case 2:
+          record.putValue(BOOLEAN_COLUMN, 0L);

Review comment:
       (nit) might be helpful to add few notes on semantics of BOOLEAN in FieldSpec class just like you have handled multiple different formats here.
   
   Is it correct to say that any signed numerical value > 0 is interpreted as true (physically stored as 1) and everything else as false (physically stored as 0)  ?
   
   Also, for source string values, will record reader (or some code in segment generator) throw exception if the source string value for a BOOLEAN data type column is anything apart from "true" or "false" ?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r618656194



##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/utils/TimestampUtils.java
##########
@@ -0,0 +1,65 @@
+/**
+ * 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.pinot.spi.utils;
+
+import java.sql.Timestamp;
+
+
+public class TimestampUtils {
+
+  /**
+   * Parses the given timestamp string into {@link Timestamp}.
+   * <p>Two formats of timestamp are supported:
+   * <ul>
+   *   <li>'yyyy-mm-dd hh:mm:ss[.fffffffff]'</li>
+   *   <li>Millis since epoch</li>
+   * </ul>
+   */
+  public static Timestamp toTimestamp(String timestampString) {
+    try {
+      return Timestamp.valueOf(timestampString);
+    } catch (Exception e) {
+      try {
+        return new Timestamp(Long.parseLong(timestampString));
+      } catch (Exception e1) {
+        throw new IllegalArgumentException(String.format("Invalid timestamp: '%s'", timestampString));
+      }
+    }
+  }
+
+  /**
+   * Parses the given timestamp string into millis since epoch.
+   * <p>Two formats of timestamp are supported:
+   * <ul>
+   *   <li>'yyyy-mm-dd hh:mm:ss[.fffffffff]'</li>
+   *   <li>Millis since epoch</li>
+   * </ul>
+   */
+  public static long toMillisSinceEpoch(String timestampString) {
+    try {
+      return Timestamp.valueOf(timestampString).getTime();
+    } catch (Exception e) {
+      try {
+        return Long.parseLong(timestampString);
+      } catch (Exception e1) {
+        throw new IllegalArgumentException(String.format("Invalid timestamp: '%s'", timestampString));

Review comment:
       Same here




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang merged pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang merged pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719


   


-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r614498498



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/operator/filter/predicate/EqualsPredicateEvaluatorFactory.java
##########
@@ -54,30 +57,36 @@ public static BaseDictionaryBasedPredicateEvaluator newDictionaryBasedEvaluator(
    */
   public static BaseRawValueBasedPredicateEvaluator newRawValueBasedEvaluator(EqPredicate eqPredicate,
       DataType dataType) {
+    String value = eqPredicate.getValue();
     switch (dataType) {
       case INT:
-        return new IntRawValueBasedEqPredicateEvaluator(eqPredicate);
+        return new IntRawValueBasedEqPredicateEvaluator(Integer.parseInt(value));

Review comment:
       Yes, because `BOOLEAN` and `TIMESTAMP` is stored as another type, making these changes so that the code can be reused.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] codecov-commenter edited a comment on pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#issuecomment-827242765


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#6719](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (989db05) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/0dcdc5b5beb928b61d54c88d9b9051d23f656d77?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0dcdc5b) will **increase** coverage by `0.10%`.
   > The diff coverage is `59.55%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6719/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6719      +/-   ##
   ============================================
   + Coverage     73.93%   74.03%   +0.10%     
     Complexity       12       12              
   ============================================
     Files          1421     1424       +3     
     Lines         69142    69348     +206     
     Branches       9987    10017      +30     
   ============================================
   + Hits          51118    51341     +223     
   + Misses        14668    14649      -19     
   - Partials       3356     3358       +2     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | integration | `43.52% <33.73%> (+0.05%)` | `7.00 <0.00> (ø)` | |
   | unittests | `66.01% <58.08%> (+0.06%)` | `12.00 <0.00> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...roker/requesthandler/BaseBrokerRequestHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcmVxdWVzdGhhbmRsZXIvQmFzZUJyb2tlclJlcXVlc3RIYW5kbGVyLmphdmE=) | `75.43% <0.00%> (+0.28%)` | `0.00 <0.00> (ø)` | |
   | [...or/dociditerators/ExpressionScanDocIdIterator.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9kb2NpZGl0ZXJhdG9ycy9FeHByZXNzaW9uU2NhbkRvY0lkSXRlcmF0b3IuamF2YQ==) | `67.11% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...core/operator/docvalsets/TransformBlockValSet.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9kb2N2YWxzZXRzL1RyYW5zZm9ybUJsb2NrVmFsU2V0LmphdmE=) | `23.52% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...predicate/RegexpLikePredicateEvaluatorFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9maWx0ZXIvcHJlZGljYXRlL1JlZ2V4cExpa2VQcmVkaWNhdGVFdmFsdWF0b3JGYWN0b3J5LmphdmE=) | `26.92% <0.00%> (-1.08%)` | `0.00 <0.00> (ø)` | |
   | [...nsform/function/ArrayAverageTransformFunction.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vQXJyYXlBdmVyYWdlVHJhbnNmb3JtRnVuY3Rpb24uamF2YQ==) | `43.47% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...ansform/function/ArrayLengthTransformFunction.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vQXJyYXlMZW5ndGhUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `61.11% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [.../transform/function/ArrayMaxTransformFunction.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vQXJyYXlNYXhUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `27.77% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [.../transform/function/ArrayMinTransformFunction.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vQXJyYXlNaW5UcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `27.77% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [.../transform/function/ArraySumTransformFunction.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vQXJyYXlTdW1UcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `86.36% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...m/function/JsonExtractScalarTransformFunction.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vSnNvbkV4dHJhY3RTY2FsYXJUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `50.31% <ø> (ø)` | `0.00 <0.00> (ø)` | |
   | ... and [155 more](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [0dcdc5b...989db05](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620659388



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/scalar/DateTimeFunctions.java
##########
@@ -228,6 +229,22 @@ public static long fromEpochDaysBucket(long days, long bucket) {
     return TimeUnit.DAYS.toMillis(days * bucket);
   }
 
+  /**

Review comment:
       There are existing functions in this class like dayOfWeek, dayOfYear etc. I think we should add a version of those on Timestamp type. Alternately, since the stored type is long, may be they will automatically work on column of TIMESTAMP type ?
   

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/scalar/DateTimeFunctions.java
##########
@@ -228,6 +229,22 @@ public static long fromEpochDaysBucket(long days, long bucket) {
     return TimeUnit.DAYS.toMillis(days * bucket);
   }
 
+  /**

Review comment:
       There are existing functions in this class like dayOfWeek, dayOfYear etc. I think we should add a version of those on TIMESTAMP type. Alternately, since the stored type is long, may be they will automatically work on column of TIMESTAMP type ?
   

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/scalar/DateTimeFunctions.java
##########
@@ -228,6 +229,22 @@ public static long fromEpochDaysBucket(long days, long bucket) {
     return TimeUnit.DAYS.toMillis(days * bucket);
   }
 
+  /**

Review comment:
       There are existing functions in this class like dayOfWeek, dayOfYear etc. I think we should add a version of those on `TIMESTAMP` type. Alternately, since the stored type is long, may be they will automatically work on column of `TIMESTAMP` type ?
   




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] codecov-commenter commented on pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#issuecomment-827242765


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#6719](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (989db05) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/0dcdc5b5beb928b61d54c88d9b9051d23f656d77?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0dcdc5b) will **decrease** coverage by `7.91%`.
   > The diff coverage is `58.08%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6719/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6719      +/-   ##
   ============================================
   - Coverage     73.93%   66.01%   -7.92%     
     Complexity       12       12              
   ============================================
     Files          1421     1424       +3     
     Lines         69142    69348     +206     
     Branches       9987    10017      +30     
   ============================================
   - Hits          51118    45778    -5340     
   - Misses        14668    20320    +5652     
   + Partials       3356     3250     -106     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | integration | `?` | `?` | |
   | unittests | `66.01% <58.08%> (+0.06%)` | `12.00 <0.00> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...roker/requesthandler/BaseBrokerRequestHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcmVxdWVzdGhhbmRsZXIvQmFzZUJyb2tlclJlcXVlc3RIYW5kbGVyLmphdmE=) | `19.65% <0.00%> (-55.50%)` | `0.00 <0.00> (ø)` | |
   | [...che/pinot/core/common/datatable/BaseDataTable.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9jb21tb24vZGF0YXRhYmxlL0Jhc2VEYXRhVGFibGUuamF2YQ==) | `81.06% <0.00%> (-18.18%)` | `0.00 <0.00> (ø)` | |
   | [...rg/apache/pinot/core/minion/RawIndexConverter.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9taW5pb24vUmF3SW5kZXhDb252ZXJ0ZXIuamF2YQ==) | `0.00% <0.00%> (-56.61%)` | `0.00 <0.00> (ø)` | |
   | [...or/dociditerators/ExpressionScanDocIdIterator.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9kb2NpZGl0ZXJhdG9ycy9FeHByZXNzaW9uU2NhbkRvY0lkSXRlcmF0b3IuamF2YQ==) | `46.30% <0.00%> (-20.81%)` | `0.00 <0.00> (ø)` | |
   | [...core/operator/docvalsets/TransformBlockValSet.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9kb2N2YWxzZXRzL1RyYW5zZm9ybUJsb2NrVmFsU2V0LmphdmE=) | `21.56% <0.00%> (-1.97%)` | `0.00 <0.00> (ø)` | |
   | [...predicate/RegexpLikePredicateEvaluatorFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9maWx0ZXIvcHJlZGljYXRlL1JlZ2V4cExpa2VQcmVkaWNhdGVFdmFsdWF0b3JGYWN0b3J5LmphdmE=) | `26.92% <0.00%> (-1.08%)` | `0.00 <0.00> (ø)` | |
   | [...nsform/function/ArrayAverageTransformFunction.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vQXJyYXlBdmVyYWdlVHJhbnNmb3JtRnVuY3Rpb24uamF2YQ==) | `43.47% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...ansform/function/ArrayLengthTransformFunction.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vQXJyYXlMZW5ndGhUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `47.22% <0.00%> (-13.89%)` | `0.00 <0.00> (ø)` | |
   | [.../transform/function/ArrayMaxTransformFunction.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vQXJyYXlNYXhUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `27.77% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [.../transform/function/ArrayMinTransformFunction.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vQXJyYXlNaW5UcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `27.77% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | ... and [459 more](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [0dcdc5b...989db05](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620647615



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/TimestampQueriesTest.java
##########
@@ -0,0 +1,229 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.sql.Timestamp;

Review comment:
       It looks like we should use java.time.*
   
   This article says joda time, java.sql.time* and all of old time/date classes usage should be migrated to use java.time
   
   https://softwareengineering.stackexchange.com/questions/190891/joda-time-vs-java-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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620728733



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java
##########
@@ -235,33 +234,32 @@ public void init(SegmentGeneratorConfig segmentCreationSpec, SegmentIndexCreatio
         Preconditions.checkState(!invertedIndexColumns.contains(columnName),
             "Cannot create inverted index for raw index column: %s", columnName);
 
-        ChunkCompressionType compressionType =
-            getColumnCompressionType(segmentCreationSpec, fieldSpec);
+        ChunkCompressionType compressionType = getColumnCompressionType(segmentCreationSpec, fieldSpec);
 
         // Initialize forward index creator
         boolean deriveNumDocsPerChunk =
             shouldDeriveNumDocsPerChunk(columnName, segmentCreationSpec.getColumnProperties());
         int writerVersion = rawIndexWriterVersion(columnName, segmentCreationSpec.getColumnProperties());
         _forwardIndexCreatorMap.put(columnName,
-            getRawIndexCreatorForColumn(_indexDir, compressionType, columnName, fieldSpec.getDataType(), totalDocs,
+            getRawIndexCreatorForColumn(_indexDir, compressionType, columnName, storedType, totalDocs,
                 indexCreationInfo.getLengthOfLongestEntry(), deriveNumDocsPerChunk, writerVersion));
       }
 
       if (textIndexColumns.contains(columnName)) {
         // Initialize text index creator
         Preconditions.checkState(fieldSpec.isSingleValueField(),
             "Text index is currently only supported on single-value columns");
-        Preconditions.checkState(fieldSpec.getDataType() == STRING,
-            "Text index is currently only supported on STRING type columns");
+        Preconditions
+            .checkState(storedType == DataType.STRING, "Text index is currently only supported on STRING type columns");

Review comment:
       We can add more logical type check in the future, but for existing checks I keep it physical type check so that it is backward-compatible




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] codecov-io commented on pull request #6719: [WIP] Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
codecov-io commented on pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#issuecomment-812265619


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=h1) Report
   > Merging [#6719](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=desc) (3fec2c1) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/71463492749607314b4ae171b0c02e7bbac74335?el=desc) (7146349) will **decrease** coverage by `7.82%`.
   > The diff coverage is `56.76%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6719/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6719      +/-   ##
   ============================================
   - Coverage     73.80%   65.97%   -7.83%     
     Complexity       12       12              
   ============================================
     Files          1402     1406       +4     
     Lines         68082    68383     +301     
     Branches       9836     9887      +51     
   ============================================
   - Hits          50245    45117    -5128     
   - Misses        14576    20069    +5493     
   + Partials       3261     3197      -64     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | integration | `?` | `?` | |
   | unittests | `65.97% <56.76%> (-0.05%)` | `0.00 <0.00> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...roker/requesthandler/BaseBrokerRequestHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcmVxdWVzdGhhbmRsZXIvQmFzZUJyb2tlclJlcXVlc3RIYW5kbGVyLmphdmE=) | `19.68% <0.00%> (-55.72%)` | `0.00 <0.00> (ø)` | |
   | [...exception/SchemaBackwardIncompatibleException.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL1NjaGVtYUJhY2t3YXJkSW5jb21wYXRpYmxlRXhjZXB0aW9uLmphdmE=) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...inot/common/function/scalar/DateTimeFunctions.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vc2NhbGFyL0RhdGVUaW1lRnVuY3Rpb25zLmphdmE=) | `94.36% <0.00%> (-2.74%)` | `0.00 <0.00> (ø)` | |
   | [.../pinot/common/function/scalar/StringFunctions.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vc2NhbGFyL1N0cmluZ0Z1bmN0aW9ucy5qYXZh) | `69.76% <ø> (ø)` | `0.00 <0.00> (ø)` | |
   | [...ller/api/resources/PinotSchemaRestletResource.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL1Bpbm90U2NoZW1hUmVzdGxldFJlc291cmNlLmphdmE=) | `38.73% <0.00%> (-0.72%)` | `0.00 <0.00> (ø)` | |
   | [...e/pinot/core/common/datatable/DataTableImplV2.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9jb21tb24vZGF0YXRhYmxlL0RhdGFUYWJsZUltcGxWMi5qYXZh) | `89.10% <0.00%> (-0.35%)` | `0.00 <0.00> (ø)` | |
   | [...data/readers/MultiplePinotSegmentRecordReader.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3JlYWRlcnMvTXVsdGlwbGVQaW5vdFNlZ21lbnRSZWNvcmRSZWFkZXIuamF2YQ==) | `70.90% <0.00%> (-1.32%)` | `0.00 <0.00> (ø)` | |
   | [...rg/apache/pinot/core/minion/RawIndexConverter.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9taW5pb24vUmF3SW5kZXhDb252ZXJ0ZXIuamF2YQ==) | `0.00% <0.00%> (-56.61%)` | `0.00 <0.00> (ø)` | |
   | [...or/dociditerators/ExpressionScanDocIdIterator.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9kb2NpZGl0ZXJhdG9ycy9FeHByZXNzaW9uU2NhbkRvY0lkSXRlcmF0b3IuamF2YQ==) | `46.30% <0.00%> (-20.81%)` | `0.00 <0.00> (ø)` | |
   | [...core/operator/docvalsets/TransformBlockValSet.java](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9kb2N2YWxzZXRzL1RyYW5zZm9ybUJsb2NrVmFsU2V0LmphdmE=) | `21.56% <0.00%> (-1.97%)` | `0.00 <0.00> (ø)` | |
   | ... and [469 more](https://codecov.io/gh/apache/incubator-pinot/pull/6719/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=footer). Last update [7146349...3fec2c1](https://codecov.io/gh/apache/incubator-pinot/pull/6719?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r614489159



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/DataSchema.java
##########
@@ -215,7 +236,34 @@ public int hashCode() {
   }
 
   public enum ColumnDataType {
-    INT, LONG, FLOAT, DOUBLE, STRING, BYTES, OBJECT, INT_ARRAY, LONG_ARRAY, FLOAT_ARRAY, DOUBLE_ARRAY, STRING_ARRAY;
+    INT,
+    LONG,
+    FLOAT,
+    DOUBLE,
+    BOOLEAN /* Stored as INT */,

Review comment:
       Internally we don't have `BYTE` type, so here we use the smallest type to store `BOOLEAN`.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r618867424



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -315,6 +408,58 @@ public Double convert(Object value, PinotDataType sourceType) {
     }
   },
 
+  TIMESTAMP {
+    @Override
+    public int toInt(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from TIMESTAMP to INTEGER");
+    }
+
+    @Override
+    public long toLong(Object value) {
+      return ((Timestamp) value).getTime();
+    }
+
+    @Override
+    public float toFloat(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from TIMESTAMP to FLOAT");
+    }
+
+    @Override
+    public double toDouble(Object value) {
+      return ((Timestamp) value).getTime();
+    }
+
+    @Override
+    public boolean toBoolean(Object value) {
+      return ((Timestamp) value).getTime() != 0;

Review comment:
       This method will be triggered when the input value is of type `Timestamp`, but the schema defines the field as `BOOLEAN`. In regular cases it should not be used.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] amrishlal commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
amrishlal commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r612097293



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/DataSchema.java
##########
@@ -215,7 +236,34 @@ public int hashCode() {
   }
 
   public enum ColumnDataType {
-    INT, LONG, FLOAT, DOUBLE, STRING, BYTES, OBJECT, INT_ARRAY, LONG_ARRAY, FLOAT_ARRAY, DOUBLE_ARRAY, STRING_ARRAY;
+    INT,
+    LONG,
+    FLOAT,
+    DOUBLE,
+    BOOLEAN /* Stored as INT */,

Review comment:
       Should `BOOLEAN` should be stored as `BYTE` to save space?

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -92,6 +115,16 @@ public double toDouble(Object value) {
       return ((Byte) value).doubleValue();
     }
 
+    @Override
+    public boolean toBoolean(Object value) {
+      return (Byte) value > 0;

Review comment:
       Can we replace this by `return (Byte) value != 0`? I think the common convention (at least in c/c++ and a few other languages) is to treat zero as false and everything else as true. Same with other data types (character, integer, long, float) toBoolean function.

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -315,6 +408,58 @@ public Double convert(Object value, PinotDataType sourceType) {
     }
   },
 
+  TIMESTAMP {
+    @Override
+    public int toInt(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from TIMESTAMP to INTEGER");
+    }
+
+    @Override
+    public long toLong(Object value) {
+      return ((Timestamp) value).getTime();
+    }
+
+    @Override
+    public float toFloat(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from TIMESTAMP to FLOAT");
+    }
+
+    @Override
+    public double toDouble(Object value) {
+      return ((Timestamp) value).getTime();
+    }
+
+    @Override
+    public boolean toBoolean(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from TIMESTAMP to BOOLEAN");

Review comment:
       I would have done zero as `false` and all other timestamp values as `true`. This convention generalizes to work with everything and type conversion becomes simple :-)

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -375,6 +530,16 @@ public double toDouble(Object value) {
       throw new UnsupportedOperationException("Cannot convert value from BYTES to DOUBLE");
     }
 
+    @Override
+    public boolean toBoolean(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from BYTES to BOOLEAN");

Review comment:
       If BYTES is not 0, it is true; otherwise, false?

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -412,6 +577,16 @@ public double toDouble(Object value) {
       return ((Number) value).doubleValue();
     }
 
+    @Override
+    public boolean toBoolean(Object value) {
+      return ((Number) value).intValue() > 0;

Review comment:
       Can object be null? Would null Object be considered true or false? What if we add NULL value support later?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r614491358



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -412,6 +577,16 @@ public double toDouble(Object value) {
       return ((Number) value).doubleValue();
     }
 
+    @Override
+    public boolean toBoolean(Object value) {
+      return ((Number) value).intValue() > 0;

Review comment:
       We won't get `null` here. (Usually we annotation arguments that can be `null` with `Nullable`)




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#issuecomment-825321407


   > This is certainly good work. Both BOOLEAN and TIMESTAMP would be very useful. I haven't looked at all the files yet, but will look again later. Not sure if I missed them, but would like to see more SQL level test cases for these two data types.
   
   @amrishlal Added `BooleanQueriesTest` and `TimestampQueriesTest` to test the query flow for various types of queries


-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620727747



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/BooleanQueriesTest.java
##########
@@ -0,0 +1,249 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+
+
+/**
+ * Queries test for BOOLEAN data type.
+ */
+public class BooleanQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "BooleanQueriesTest");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final int NUM_RECORDS = 1000;
+
+  private static final String BOOLEAN_COLUMN = "booleanColumn";
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(BOOLEAN_COLUMN, DataType.BOOLEAN).build();
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  @Override
+  protected String getFilter() {
+    return "";
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+
+    List<GenericRow> records = new ArrayList<>(NUM_RECORDS);
+    for (int i = 0; i < NUM_RECORDS; i++) {
+      GenericRow record = new GenericRow();
+      // Insert data in 6 different formats
+      switch (i % 6) {
+        case 0:
+          record.putValue(BOOLEAN_COLUMN, false);
+          break;
+        case 1:
+          record.putValue(BOOLEAN_COLUMN, 1);
+          break;
+        case 2:
+          record.putValue(BOOLEAN_COLUMN, 0L);

Review comment:
       Added the value conversion rules into the `PinotDataType` class javadoc




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r614490551



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -315,6 +408,58 @@ public Double convert(Object value, PinotDataType sourceType) {
     }
   },
 
+  TIMESTAMP {
+    @Override
+    public int toInt(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from TIMESTAMP to INTEGER");
+    }
+
+    @Override
+    public long toLong(Object value) {
+      return ((Timestamp) value).getTime();
+    }
+
+    @Override
+    public float toFloat(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from TIMESTAMP to FLOAT");
+    }
+
+    @Override
+    public double toDouble(Object value) {
+      return ((Timestamp) value).getTime();
+    }
+
+    @Override
+    public boolean toBoolean(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from TIMESTAMP to BOOLEAN");

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620745676



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/scalar/DateTimeFunctions.java
##########
@@ -228,6 +229,22 @@ public static long fromEpochDaysBucket(long days, long bucket) {
     return TimeUnit.DAYS.toMillis(days * bucket);
   }
 
+  /**

Review comment:
       Yes, it automatically works, and that's one reason why TIMESTAMP is stored as millis since epoch




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r618864831



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/DataSchema.java
##########
@@ -215,7 +236,34 @@ public int hashCode() {
   }
 
   public enum ColumnDataType {
-    INT, LONG, FLOAT, DOUBLE, STRING, BYTES, OBJECT, INT_ARRAY, LONG_ARRAY, FLOAT_ARRAY, DOUBLE_ARRAY, STRING_ARRAY;
+    INT,
+    LONG,
+    FLOAT,
+    DOUBLE,
+    BOOLEAN /* Stored as INT */,

Review comment:
       Probably no. It doesn't make sense to add them as logical types and backed by int because that won't save any storage.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620638226



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java
##########
@@ -235,33 +234,32 @@ public void init(SegmentGeneratorConfig segmentCreationSpec, SegmentIndexCreatio
         Preconditions.checkState(!invertedIndexColumns.contains(columnName),
             "Cannot create inverted index for raw index column: %s", columnName);
 
-        ChunkCompressionType compressionType =
-            getColumnCompressionType(segmentCreationSpec, fieldSpec);
+        ChunkCompressionType compressionType = getColumnCompressionType(segmentCreationSpec, fieldSpec);
 
         // Initialize forward index creator
         boolean deriveNumDocsPerChunk =
             shouldDeriveNumDocsPerChunk(columnName, segmentCreationSpec.getColumnProperties());
         int writerVersion = rawIndexWriterVersion(columnName, segmentCreationSpec.getColumnProperties());
         _forwardIndexCreatorMap.put(columnName,
-            getRawIndexCreatorForColumn(_indexDir, compressionType, columnName, fieldSpec.getDataType(), totalDocs,
+            getRawIndexCreatorForColumn(_indexDir, compressionType, columnName, storedType, totalDocs,
                 indexCreationInfo.getLengthOfLongestEntry(), deriveNumDocsPerChunk, writerVersion));
       }
 
       if (textIndexColumns.contains(columnName)) {
         // Initialize text index creator
         Preconditions.checkState(fieldSpec.isSingleValueField(),
             "Text index is currently only supported on single-value columns");
-        Preconditions.checkState(fieldSpec.getDataType() == STRING,
-            "Text index is currently only supported on STRING type columns");
+        Preconditions
+            .checkState(storedType == DataType.STRING, "Text index is currently only supported on STRING type columns");

Review comment:
       With the current ongoing discussion for nested/json types, flattening etc, the short term will have JSON data type and the forward index will continue to have it as STRING as stored type. 
   The above check will actually pass for a json column if the user wants to create a text index on it. I don't think we want that at least until we have tested how text index will work on json (stringified ) data. I think the check should still happen using the logical type and not the stored type
   cc @amrishlal 




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620672041



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/TimestampQueriesTest.java
##########
@@ -0,0 +1,229 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.sql.Timestamp;

Review comment:
       `Both Joda-Time and the original Java date-time classes are now outmoded by the java.time classes built into Java 8, Java 9, and later.` from the thread. Looks like we should be using java.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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620657416



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/scalar/DateTimeFunctions.java
##########
@@ -228,6 +229,22 @@ public static long fromEpochDaysBucket(long days, long bucket) {
     return TimeUnit.DAYS.toMillis(days * bucket);
   }
 
+  /**
+   * Converts epoch millis to Timestamp
+   */
+  @ScalarFunction
+  public static Timestamp toTimestamp(long millis) {
+    return new Timestamp(millis);
+  }
+
+  /**
+   * Converts Timestamp to epoch millis
+   */
+  @ScalarFunction
+  public static long fromTimestamp(Timestamp timestamp) {

Review comment:
       Also, what about adding other utility functions like adding/subtracting two timestamps. I think users can probably use ADD/SUB non-scalar transform function on TIMESTAMP type but probably this is a better place




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r614490111



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -92,6 +115,16 @@ public double toDouble(Object value) {
       return ((Byte) value).doubleValue();
     }
 
+    @Override
+    public boolean toBoolean(Object value) {
+      return (Byte) value > 0;

Review comment:
       Good suggestion, changed




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] amrishlal commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
amrishlal commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r612097293



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/DataSchema.java
##########
@@ -215,7 +236,34 @@ public int hashCode() {
   }
 
   public enum ColumnDataType {
-    INT, LONG, FLOAT, DOUBLE, STRING, BYTES, OBJECT, INT_ARRAY, LONG_ARRAY, FLOAT_ARRAY, DOUBLE_ARRAY, STRING_ARRAY;
+    INT,
+    LONG,
+    FLOAT,
+    DOUBLE,
+    BOOLEAN /* Stored as INT */,

Review comment:
       Should `BOOLEAN` be stored as `BYTE` to save space?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620743637



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/TimestampQueriesTest.java
##########
@@ -0,0 +1,229 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+
+
+/**
+ * Queries test for TIMESTAMP data type.
+ */
+public class TimestampQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "TimestampQueriesTest");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+  private static final long BASE_TIMESTAMP = Timestamp.valueOf("2021-01-01 00:00:00").getTime();

Review comment:
       Added more notes in `PinotDataType` which handles the type conversion.
   We always store TIMESTAMP in millis since epoch. I don't see much value keeping micros/nanos granularity, and millis is the smallest granularity for time management in pinot




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] amrishlal commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
amrishlal commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r612097546



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/operator/filter/predicate/EqualsPredicateEvaluatorFactory.java
##########
@@ -54,30 +57,36 @@ public static BaseDictionaryBasedPredicateEvaluator newDictionaryBasedEvaluator(
    */
   public static BaseRawValueBasedPredicateEvaluator newRawValueBasedEvaluator(EqPredicate eqPredicate,
       DataType dataType) {
+    String value = eqPredicate.getValue();
     switch (dataType) {
       case INT:
-        return new IntRawValueBasedEqPredicateEvaluator(eqPredicate);
+        return new IntRawValueBasedEqPredicateEvaluator(Integer.parseInt(value));

Review comment:
       Are these changes related to BOOLEAN and TIMESTAMP support?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] amrishlal commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
amrishlal commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r618121047



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java
##########
@@ -737,11 +732,11 @@ public static void removeColumnMetadataInfo(PropertiesConfiguration properties,
    * @return raw index creator
    * @throws IOException
    */
-  public static ForwardIndexCreator getRawIndexCreatorForColumn(File file,
-      ChunkCompressionType compressionType, String column, DataType dataType, int totalDocs,
-      int lengthOfLongestEntry, boolean deriveNumDocsPerChunk, int writerVersion)
+  public static ForwardIndexCreator getRawIndexCreatorForColumn(File file, ChunkCompressionType compressionType,
+      String column, DataType dataType, int totalDocs, int lengthOfLongestEntry, boolean deriveNumDocsPerChunk,
+      int writerVersion)
       throws IOException {
-    switch (dataType) {
+    switch (dataType.getStoredType()) {

Review comment:
       Can you please add some comments in DataType.java explaining the usage DataType vs stored DataType, i.e when should DataType be used and when should DataType.getStoredType() be used?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r618865977



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -60,6 +63,16 @@ public double toDouble(Object value) {
       return ((Boolean) value) ? 1d : 0d;
     }
 
+    @Override
+    public boolean toBoolean(Object value) {

Review comment:
       We won't allow `null` value inside this type conversion util class. `null` value should be handled outside.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] amrishlal commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
amrishlal commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r618120670



##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/utils/TimestampUtils.java
##########
@@ -0,0 +1,65 @@
+/**
+ * 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.pinot.spi.utils;
+
+import java.sql.Timestamp;
+
+
+public class TimestampUtils {
+
+  /**
+   * Parses the given timestamp string into {@link Timestamp}.
+   * <p>Two formats of timestamp are supported:
+   * <ul>
+   *   <li>'yyyy-mm-dd hh:mm:ss[.fffffffff]'</li>
+   *   <li>Millis since epoch</li>
+   * </ul>
+   */
+  public static Timestamp toTimestamp(String timestampString) {
+    try {
+      return Timestamp.valueOf(timestampString);
+    } catch (Exception e) {
+      try {
+        return new Timestamp(Long.parseLong(timestampString));
+      } catch (Exception e1) {
+        throw new IllegalArgumentException(String.format("Invalid timestamp: '%s'", timestampString));
+      }
+    }
+  }
+
+  /**
+   * Parses the given timestamp string into millis since epoch.
+   * <p>Two formats of timestamp are supported:
+   * <ul>
+   *   <li>'yyyy-mm-dd hh:mm:ss[.fffffffff]'</li>
+   *   <li>Millis since epoch</li>
+   * </ul>
+   */
+  public static long toMillisSinceEpoch(String timestampString) {
+    try {
+      return Timestamp.valueOf(timestampString).getTime();
+    } catch (Exception e) {
+      try {
+        return Long.parseLong(timestampString);
+      } catch (Exception e1) {
+        throw new IllegalArgumentException(String.format("Invalid timestamp: '%s'", timestampString));

Review comment:
       Same here. Would like to see root exception being included.

##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/utils/TimestampUtils.java
##########
@@ -0,0 +1,65 @@
+/**
+ * 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.pinot.spi.utils;
+
+import java.sql.Timestamp;
+
+
+public class TimestampUtils {
+
+  /**
+   * Parses the given timestamp string into {@link Timestamp}.
+   * <p>Two formats of timestamp are supported:
+   * <ul>
+   *   <li>'yyyy-mm-dd hh:mm:ss[.fffffffff]'</li>
+   *   <li>Millis since epoch</li>
+   * </ul>
+   */
+  public static Timestamp toTimestamp(String timestampString) {
+    try {
+      return Timestamp.valueOf(timestampString);
+    } catch (Exception e) {
+      try {
+        return new Timestamp(Long.parseLong(timestampString));
+      } catch (Exception e1) {
+        throw new IllegalArgumentException(String.format("Invalid timestamp: '%s'", timestampString));

Review comment:
       Can we also include the root exception in IllegalArgumentException. Makes it easier to trace back to the root cause or exception sequence.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620743637



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/TimestampQueriesTest.java
##########
@@ -0,0 +1,229 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataSchema.ColumnDataType;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+
+
+/**
+ * Queries test for TIMESTAMP data type.
+ */
+public class TimestampQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "TimestampQueriesTest");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+  private static final long BASE_TIMESTAMP = Timestamp.valueOf("2021-01-01 00:00:00").getTime();

Review comment:
       Added more notes in `PinotDataType` which handles the type conversion.
   We always store TIMESTAMP in millis since epoch. I don't see much value keeping micros/nanos granularity, and millis is the smallest granularity for time management in pinot.
   Timezone is not handled within the TIMESTAMP type. We will add a separate type (probably DATETIME) for timezone handling




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] amrishlal commented on pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
amrishlal commented on pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#issuecomment-825398934


   > @amrishlal Added `BooleanQueriesTest` and `TimestampQueriesTest` to test the query flow for various types of queries
   
   I would have probably used 20 records instead of 1000, but looks good to me :-)
   


-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620745364



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/function/scalar/DateTimeFunctions.java
##########
@@ -228,6 +229,22 @@ public static long fromEpochDaysBucket(long days, long bucket) {
     return TimeUnit.DAYS.toMillis(days * bucket);
   }
 
+  /**
+   * Converts epoch millis to Timestamp
+   */
+  @ScalarFunction
+  public static Timestamp toTimestamp(long millis) {
+    return new Timestamp(millis);
+  }
+
+  /**
+   * Converts Timestamp to epoch millis
+   */
+  @ScalarFunction
+  public static long fromTimestamp(Timestamp timestamp) {

Review comment:
       This follows the naming convention of other functions (e.g. `fromDateTime`, `fromEpochDays` etc.)
   We can add more util functions in the following PRs as needed. That should be very straight forward




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6719: Add TIMESTAMP and BOOLEAN data type support

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6719:
URL: https://github.com/apache/incubator-pinot/pull/6719#discussion_r620746269



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -315,6 +408,58 @@ public Double convert(Object value, PinotDataType sourceType) {
     }
   },
 
+  TIMESTAMP {
+    @Override
+    public int toInt(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from TIMESTAMP to INTEGER");
+    }
+
+    @Override
+    public long toLong(Object value) {
+      return ((Timestamp) value).getTime();
+    }
+
+    @Override
+    public float toFloat(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from TIMESTAMP to FLOAT");
+    }
+
+    @Override
+    public double toDouble(Object value) {
+      return ((Timestamp) value).getTime();
+    }
+
+    @Override
+    public boolean toBoolean(Object value) {
+      throw new UnsupportedOperationException("Cannot convert value from TIMESTAMP to BOOLEAN");

Review comment:
       Revert this to unsupported because I feel it makes no sense to convert a timestamp to boolean, most likely a human error of putting the wrong data type




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org