You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ne...@apache.org on 2020/05/08 19:09:34 UTC

[incubator-pinot] branch master updated: Remove excessive SchemaBuilder#addTime methods (#5349)

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

nehapawar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new aa0a5f1  Remove excessive SchemaBuilder#addTime methods (#5349)
aa0a5f1 is described below

commit aa0a5f144ae360ef447dea01c5cc949e1b327989
Author: Neha Pawar <ne...@gmail.com>
AuthorDate: Fri May 8 12:09:20 2020 -0700

    Remove excessive SchemaBuilder#addTime methods (#5349)
    
    When timeFieldSpec gets deprecated in favor of DateTimeFieldSpec, one of the changes will be in the SchemaBuilder class. SchemaBuilder is used only in tests (and some external integrations maybe).
    SchemaBuilder has ~10 methods to addTime. These will be replaced with addDateTime (barring a few places where we intentionally want to add TIME to test that we get DATE_TIME.) Removing most of the addTime methods, keeping only one
---
 .../broker/broker/HelixBrokerStarterTest.java      |   3 +-
 .../timeboundary/TimeBoundaryManagerTest.java      |   4 +-
 .../org/apache/pinot/common/utils/SchemaUtils.java |   3 +-
 .../apache/pinot/common/data/FieldSpecTest.java    |   9 +-
 .../org/apache/pinot/common/data/SchemaTest.java   |  83 ++------------
 ...PinotInstanceAssignmentRestletResourceTest.java |   5 +-
 .../data/readers/RecordReaderSampleDataTest.java   |  97 ++---------------
 .../ExpressionTransformerTimeTest.java             |   4 +-
 .../generator/SegmentGeneratorConfigTest.java      |   4 +-
 .../MutableSegmentImplAggregateMetricsTest.java    |   4 +-
 ...adataAndDictionaryAggregationPlanMakerTest.java |   5 +-
 .../pinot/queries/BaseMultiValueQueriesTest.java   |   4 +-
 .../pinot/queries/BaseSingleValueQueriesTest.java  |   4 +-
 .../apache/pinot/queries/FastHllQueriesTest.java   |   4 +-
 .../apache/pinot/queries/TransformQueriesTest.java |   4 +-
 .../LuceneRealtimeClusterIntegrationTest.java      |   2 +-
 .../avro/AvroRecordToPinotRowGeneratorTest.java    |   5 +-
 .../plugin/inputformat/avro/AvroUtilsTest.java     |   3 +-
 .../apache/pinot/spi/data/DateTimeFieldSpec.java   |  16 ++-
 .../java/org/apache/pinot/spi/data/FieldSpec.java  |   2 +-
 .../java/org/apache/pinot/spi/data/Schema.java     | 119 ++++++++-------------
 .../pinot/spi/data/DateTimeFieldSpecUtilsTest.java |  41 +++----
 .../tools/admin/command/GenerateDataCommand.java   |   3 +-
 23 files changed, 147 insertions(+), 281 deletions(-)

diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java
index 42f8e02..b855fb5 100644
--- a/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java
+++ b/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java
@@ -41,6 +41,7 @@ import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.config.table.TableType;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.TimeGranularitySpec;
 import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
 import org.apache.pinot.spi.utils.builder.TableNameBuilder;
 import org.apache.pinot.util.TestUtils;
@@ -80,7 +81,7 @@ public class HelixBrokerStarterTest extends ControllerTest {
     addFakeServerInstancesToAutoJoinHelixCluster(NUM_SERVERS, true);
 
     Schema schema = new Schema.SchemaBuilder().setSchemaName(RAW_TABLE_NAME)
-        .addTime(TIME_COLUMN_NAME, TimeUnit.DAYS, FieldSpec.DataType.INT).build();
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, TIME_COLUMN_NAME), null).build();
     _helixResourceManager.addSchema(schema, true);
     TableConfig offlineTableConfig =
         new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setTimeColumnName(TIME_COLUMN_NAME)
diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/timeboundary/TimeBoundaryManagerTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/timeboundary/TimeBoundaryManagerTest.java
index 280bc5f..9cca5b3 100644
--- a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/timeboundary/TimeBoundaryManagerTest.java
+++ b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/timeboundary/TimeBoundaryManagerTest.java
@@ -34,6 +34,7 @@ import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.config.table.TableType;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.TimeGranularitySpec;
 import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
 import org.apache.pinot.spi.utils.builder.TableNameBuilder;
 import org.mockito.Mockito;
@@ -148,7 +149,8 @@ public class TimeBoundaryManagerTest {
 
   private void setSchema(String rawTableName, TimeUnit timeUnit) {
     ZKMetadataProvider.setSchema(_propertyStore,
-        new Schema.SchemaBuilder().setSchemaName(rawTableName).addTime(TIME_COLUMN, timeUnit, FieldSpec.DataType.LONG)
+        new Schema.SchemaBuilder().setSchemaName(rawTableName)
+            .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, timeUnit, TIME_COLUMN), null)
             .build());
   }
 
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/SchemaUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/SchemaUtils.java
index 6b97c13..b47c0fb 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/utils/SchemaUtils.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/SchemaUtils.java
@@ -35,6 +35,7 @@ import org.apache.commons.httpclient.params.HttpMethodParams;
 import org.apache.helix.ZNRecord;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.TimeGranularitySpec;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -188,7 +189,7 @@ public class SchemaUtils {
   public static void main(String[] args) {
     Schema schema = new Schema.SchemaBuilder().setSchemaName("testSchema")
         .addSingleValueDimension("dimension", FieldSpec.DataType.DOUBLE).addMetric("metric", FieldSpec.DataType.INT)
-        .addTime("time", TimeUnit.DAYS, FieldSpec.DataType.INT).build();
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "time"), null).build();
     System.out.println(postSchema("localhost", 8100, schema));
     Schema fetchedSchema = getSchema("localhost", 8100, "testSchema");
     Preconditions.checkNotNull(fetchedSchema);
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/data/FieldSpecTest.java b/pinot-common/src/test/java/org/apache/pinot/common/data/FieldSpecTest.java
index 42cb4ce..5ac1879 100644
--- a/pinot-common/src/test/java/org/apache/pinot/common/data/FieldSpecTest.java
+++ b/pinot-common/src/test/java/org/apache/pinot/common/data/FieldSpecTest.java
@@ -213,10 +213,17 @@ public class FieldSpecTest {
 
     DateTimeFieldSpec dateTimeFieldSpec1 = new DateTimeFieldSpec(name, LONG, format, granularity);
     DateTimeFieldSpec dateTimeFieldSpec2 = new DateTimeFieldSpec(name, INT, format, granularity);
-    Assert.assertFalse(dateTimeFieldSpec1.equals(dateTimeFieldSpec2));
+    Assert.assertNotEquals(dateTimeFieldSpec2, dateTimeFieldSpec1);
 
     DateTimeFieldSpec dateTimeFieldSpec3 = new DateTimeFieldSpec(name, LONG, format, granularity);
     Assert.assertEquals(dateTimeFieldSpec1, dateTimeFieldSpec3);
+
+    DateTimeFieldSpec dateTimeFieldSpec4 = new DateTimeFieldSpec(name, LONG, format, granularity, 100000000L, null);
+    DateTimeFieldSpec dateTimeFieldSpec5 = new DateTimeFieldSpec(name, INT, format, granularity, null, "toEpochHours(millis)");
+    Assert.assertNotEquals(dateTimeFieldSpec5, dateTimeFieldSpec4);
+
+    DateTimeFieldSpec dateTimeFieldSpec6 = new DateTimeFieldSpec(name, LONG, format, granularity, 100000000L, null);
+    Assert.assertEquals(dateTimeFieldSpec4, dateTimeFieldSpec6);
   }
 
   @Test(dataProvider = "testFormatDataProvider")
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/data/SchemaTest.java b/pinot-common/src/test/java/org/apache/pinot/common/data/SchemaTest.java
index 0801368..15c7159 100644
--- a/pinot-common/src/test/java/org/apache/pinot/common/data/SchemaTest.java
+++ b/pinot-common/src/test/java/org/apache/pinot/common/data/SchemaTest.java
@@ -76,7 +76,7 @@ public class SchemaTest {
         .addMultiValueDimension("mvDimension", FieldSpec.DataType.STRING)
         .addMultiValueDimension("mvDimensionWithDefault", FieldSpec.DataType.STRING, defaultString)
         .addMetric("metric", FieldSpec.DataType.INT).addMetric("metricWithDefault", FieldSpec.DataType.INT, 5)
-        .addTime("time", TimeUnit.DAYS, FieldSpec.DataType.LONG)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.DAYS, "time"), null)
         .addDateTime("dateTime", FieldSpec.DataType.LONG, "1:HOURS:EPOCH", "1:HOURS").build();
 
     DimensionFieldSpec dimensionFieldSpec = schema.getDimensionSpec("svDimension");
@@ -162,79 +162,18 @@ public class SchemaTest {
         new TimeGranularitySpec(outgoingDataType, outgoingTimeUnitSize, outgoingTimeUnit, outgoingName);
     int defaultNullValue = 17050;
 
-    Schema schema1 =
-        new Schema.SchemaBuilder().setSchemaName("testSchema").addTime(incomingName, incomingTimeUnit, incomingDataType)
-            .build();
-    Schema schema2 = new Schema.SchemaBuilder().setSchemaName("testSchema")
-        .addTime(incomingName, incomingTimeUnit, incomingDataType, defaultNullValue).build();
-    Schema schema3 = new Schema.SchemaBuilder().setSchemaName("testSchema")
-        .addTime(incomingName, incomingTimeUnit, incomingDataType, outgoingName, outgoingTimeUnit, outgoingDataType)
-        .build();
-    Schema schema4 = new Schema.SchemaBuilder().setSchemaName("testSchema")
-        .addTime(incomingName, incomingTimeUnit, incomingDataType, outgoingName, outgoingTimeUnit, outgoingDataType,
-            defaultNullValue).build();
-    Schema schema5 = new Schema.SchemaBuilder().setSchemaName("testSchema")
-        .addTime(incomingName, incomingTimeUnitSize, incomingTimeUnit, incomingDataType).build();
-    Schema schema6 = new Schema.SchemaBuilder().setSchemaName("testSchema")
-        .addTime(incomingName, incomingTimeUnitSize, incomingTimeUnit, incomingDataType, defaultNullValue).build();
-    Schema schema7 = new Schema.SchemaBuilder().setSchemaName("testSchema")
-        .addTime(incomingName, incomingTimeUnitSize, incomingTimeUnit, incomingDataType, outgoingName,
-            outgoingTimeUnitSize, outgoingTimeUnit, outgoingDataType).build();
-    Schema schema8 = new Schema.SchemaBuilder().setSchemaName("testSchema")
-        .addTime(incomingName, incomingTimeUnitSize, incomingTimeUnit, incomingDataType, outgoingName,
-            outgoingTimeUnitSize, outgoingTimeUnit, outgoingDataType, defaultNullValue).build();
-    Schema schema9 =
-        new Schema.SchemaBuilder().setSchemaName("testSchema").addTime(incomingTimeGranularitySpec).build();
-    Schema schema10 =
-        new Schema.SchemaBuilder().setSchemaName("testSchema").addTime(incomingTimeGranularitySpec, defaultNullValue)
-            .build();
     Schema schema11 = new Schema.SchemaBuilder().setSchemaName("testSchema")
         .addTime(incomingTimeGranularitySpec, outgoingTimeGranularitySpec).build();
-    Schema schema12 = new Schema.SchemaBuilder().setSchemaName("testSchema")
-        .addTime(incomingTimeGranularitySpec, outgoingTimeGranularitySpec, defaultNullValue).build();
-
-    Assert.assertNotNull(schema1.getTimeFieldSpec());
-    Assert.assertNotNull(schema2.getTimeFieldSpec());
-    Assert.assertNotNull(schema3.getTimeFieldSpec());
-    Assert.assertNotNull(schema4.getTimeFieldSpec());
-    Assert.assertNotNull(schema5.getTimeFieldSpec());
-    Assert.assertNotNull(schema6.getTimeFieldSpec());
-    Assert.assertNotNull(schema7.getTimeFieldSpec());
-    Assert.assertNotNull(schema8.getTimeFieldSpec());
-    Assert.assertNotNull(schema9.getTimeFieldSpec());
-    Assert.assertNotNull(schema10.getTimeFieldSpec());
+    Schema schema12 = new Schema.SchemaBuilder().setSchemaName("testSchema").build();
+    schema12.addField(new TimeFieldSpec(incomingTimeGranularitySpec, outgoingTimeGranularitySpec, defaultNullValue));
     Assert.assertNotNull(schema11.getTimeFieldSpec());
     Assert.assertNotNull(schema12.getTimeFieldSpec());
 
-    Assert.assertEquals(schema1, schema5);
-    Assert.assertEquals(schema1, schema9);
-    Assert.assertEquals(schema2, schema6);
-    Assert.assertEquals(schema2, schema10);
-    Assert.assertEquals(schema3, schema7);
-    Assert.assertEquals(schema3, schema11);
-    Assert.assertEquals(schema4, schema8);
-    Assert.assertEquals(schema4, schema12);
-
     // Before adding default null value.
-    Assert.assertFalse(schema1.equals(schema2));
-    Assert.assertFalse(schema3.equals(schema4));
-    Assert.assertFalse(schema5.equals(schema6));
-    Assert.assertFalse(schema7.equals(schema8));
-    Assert.assertFalse(schema9.equals(schema10));
-    Assert.assertFalse(schema11.equals(schema12));
+    Assert.assertNotEquals(schema12, schema11);
 
     // After adding default null value.
-    schema1.getTimeFieldSpec().setDefaultNullValue(defaultNullValue);
-    schema3.getTimeFieldSpec().setDefaultNullValue(defaultNullValue);
-    schema5.getTimeFieldSpec().setDefaultNullValue(defaultNullValue);
-    schema7.getTimeFieldSpec().setDefaultNullValue(defaultNullValue);
-    schema9.getTimeFieldSpec().setDefaultNullValue(defaultNullValue);
     schema11.getTimeFieldSpec().setDefaultNullValue(defaultNullValue);
-    Assert.assertEquals(schema1, schema2);
-    Assert.assertEquals(schema3, schema4);
-    Assert.assertEquals(schema5, schema6);
-    Assert.assertEquals(schema7, schema8);
-    Assert.assertEquals(schema9, schema10);
     Assert.assertEquals(schema11, schema12);
   }
 
@@ -312,7 +251,7 @@ public class SchemaTest {
         .addMultiValueDimension("mvDimension", FieldSpec.DataType.STRING)
         .addMultiValueDimension("mvDimensionWithDefault", FieldSpec.DataType.STRING, "default")
         .addMetric("metric", FieldSpec.DataType.INT).addMetric("metricWithDefault", FieldSpec.DataType.INT, 5)
-        .addTime("time", TimeUnit.DAYS, FieldSpec.DataType.LONG)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.DAYS, "time"), null)
         .addDateTime("dateTime", FieldSpec.DataType.LONG, "1:HOURS:EPOCH", "1:HOURS").build();
 
     Assert.assertThrows(NullPointerException.class, () -> oldSchema.isBackwardCompatibleWith(null));
@@ -323,7 +262,7 @@ public class SchemaTest {
         .addMultiValueDimension("mvDimension", FieldSpec.DataType.STRING)
         .addMultiValueDimension("mvDimensionWithDefault", FieldSpec.DataType.STRING, "default")
         .addMetric("metric", FieldSpec.DataType.INT).addMetric("metricWithDefault", FieldSpec.DataType.INT, 5)
-        .addTime("time", TimeUnit.DAYS, FieldSpec.DataType.LONG)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.DAYS, "time"), null)
         .addDateTime("dateTime", FieldSpec.DataType.LONG, "1:HOURS:EPOCH", "1:HOURS").build();
     Assert.assertFalse(schema1.isBackwardCompatibleWith(oldSchema));
 
@@ -333,7 +272,7 @@ public class SchemaTest {
         .addMultiValueDimension("mvDimension", FieldSpec.DataType.STRING)
         .addMultiValueDimension("mvDimensionWithDefault", FieldSpec.DataType.STRING, "default")
         .addMetric("metric", FieldSpec.DataType.INT).addMetric("metricWithDefault", FieldSpec.DataType.INT, 5)
-        .addTime("time", TimeUnit.DAYS, FieldSpec.DataType.LONG)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.DAYS, "time"), null)
         .addDateTime("dateTime", FieldSpec.DataType.LONG, "1:HOURS:EPOCH", "1:HOURS").build();
     Assert.assertFalse(schema2.isBackwardCompatibleWith(oldSchema));
 
@@ -343,7 +282,7 @@ public class SchemaTest {
         .addMultiValueDimension("mvDimension", FieldSpec.DataType.STRING)
         .addMultiValueDimension("mvDimensionWithDefault", FieldSpec.DataType.STRING, "default")
         .addMetric("metric", FieldSpec.DataType.INT).addMetric("metricWithDefault", FieldSpec.DataType.INT, 5)
-        .addTime("time", TimeUnit.HOURS, FieldSpec.DataType.LONG) // DAYS -> HOURS
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.HOURS, "time"), null)
         .addDateTime("dateTime", FieldSpec.DataType.LONG, "1:HOURS:EPOCH", "1:HOURS").build();
     Assert.assertFalse(schema3.isBackwardCompatibleWith(oldSchema));
 
@@ -353,7 +292,7 @@ public class SchemaTest {
         .addMultiValueDimension("mvDimension", FieldSpec.DataType.STRING)
         .addMultiValueDimension("mvDimensionWithDefault", FieldSpec.DataType.STRING, "default")
         .addMetric("metric", FieldSpec.DataType.INT).addMetric("metricWithDefault", FieldSpec.DataType.INT, 5)
-        .addTime("time", TimeUnit.DAYS, FieldSpec.DataType.LONG)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.DAYS, "time"), null)
         .addDateTime("dateTime", FieldSpec.DataType.LONG, "2:HOURS:EPOCH", "1:HOURS").build();  // timeUnit 1 -> 2
     Assert.assertFalse(schema4.isBackwardCompatibleWith(oldSchema));
 
@@ -363,7 +302,7 @@ public class SchemaTest {
         .addMultiValueDimension("mvDimension", FieldSpec.DataType.STRING)
         .addMultiValueDimension("mvDimensionWithDefault", FieldSpec.DataType.STRING, "default")
         .addMetric("metric", FieldSpec.DataType.INT).addMetric("metricWithDefault", FieldSpec.DataType.INT, 5)
-        .addTime("time", TimeUnit.DAYS, FieldSpec.DataType.LONG)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.DAYS, "time"), null)
         .addDateTime("dateTime", FieldSpec.DataType.LONG, "1:HOURS:EPOCH", "1:HOURS").build();
     Assert.assertFalse(schema5.isBackwardCompatibleWith(oldSchema));
 
@@ -374,7 +313,7 @@ public class SchemaTest {
         .addMultiValueDimension("mvDimension", FieldSpec.DataType.STRING)
         .addMultiValueDimension("mvDimensionWithDefault", FieldSpec.DataType.STRING, "default")
         .addMetric("metric", FieldSpec.DataType.INT).addMetric("metricWithDefault", FieldSpec.DataType.INT, 5)
-        .addTime("time", TimeUnit.DAYS, FieldSpec.DataType.LONG)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.DAYS, "time"), null)
         .addDateTime("dateTime", FieldSpec.DataType.LONG, "1:HOURS:EPOCH", "1:HOURS").build();
     Assert.assertTrue(schema6.isBackwardCompatibleWith(oldSchema));
   }
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotInstanceAssignmentRestletResourceTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotInstanceAssignmentRestletResourceTest.java
index d4874e2..fcccc72 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotInstanceAssignmentRestletResourceTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotInstanceAssignmentRestletResourceTest.java
@@ -37,8 +37,10 @@ import org.apache.pinot.spi.config.table.assignment.InstanceAssignmentConfig;
 import org.apache.pinot.spi.config.table.assignment.InstancePartitionsType;
 import org.apache.pinot.spi.config.table.assignment.InstanceReplicaGroupPartitionConfig;
 import org.apache.pinot.spi.config.table.assignment.InstanceTagPoolConfig;
+import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.FieldSpec.DataType;
 import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.TimeGranularitySpec;
 import org.apache.pinot.spi.utils.JsonUtils;
 import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
 import org.apache.pinot.spi.utils.builder.TableNameBuilder;
@@ -79,7 +81,8 @@ public class PinotInstanceAssignmentRestletResourceTest extends ControllerTest {
   public void testInstanceAssignment()
       throws Exception {
     Schema schema =
-        new Schema.SchemaBuilder().setSchemaName(RAW_TABLE_NAME).addTime(TIME_COLUMN_NAME, TimeUnit.DAYS, DataType.INT)
+        new Schema.SchemaBuilder().setSchemaName(RAW_TABLE_NAME)
+            .addTime(new TimeGranularitySpec(DataType.INT, TimeUnit.DAYS, TIME_COLUMN_NAME), null)
             .build();
     _helixResourceManager.addSchema(schema, true);
     TableConfig offlineTableConfig =
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/readers/RecordReaderSampleDataTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/readers/RecordReaderSampleDataTest.java
index 3812029..26fb93a 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/data/readers/RecordReaderSampleDataTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/data/readers/RecordReaderSampleDataTest.java
@@ -21,8 +21,8 @@ package org.apache.pinot.core.data.readers;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Sets;
 import java.io.File;
+import java.util.HashSet;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
 import org.apache.pinot.core.data.recordtransformer.CompositeTransformer;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
@@ -55,19 +55,6 @@ public class RecordReaderSampleDataTest {
       .addSingleValueDimension("unknown_dimension", FieldSpec.DataType.STRING)
       .addMetric("met_impressionCount", FieldSpec.DataType.LONG).addMetric("unknown_metric", FieldSpec.DataType.DOUBLE)
       .build();
-  // Different incoming and outgoing time column name, should read both incoming and outgoing time
-  private final Schema SCHEMA_DIFFERENT_INCOMING_OUTGOING = new Schema.SchemaBuilder()
-      .addTime("time_day", TimeUnit.SECONDS, FieldSpec.DataType.LONG, "column2", TimeUnit.DAYS, FieldSpec.DataType.INT)
-      .build();
-  // Incoming time column does not exist in the record, should read outgoing time only
-  private final Schema SCHEMA_NO_INCOMING = new Schema.SchemaBuilder()
-      .addTime("incoming", TimeUnit.SECONDS, FieldSpec.DataType.LONG, "time_day", TimeUnit.DAYS, FieldSpec.DataType.INT)
-      .build();
-
-  // Outgoing time column does not exist in the record, should read incoming time only
-  private final Schema SCHEMA_NO_OUTGOING = new Schema.SchemaBuilder()
-      .addTime("time_day", TimeUnit.SECONDS, FieldSpec.DataType.LONG, "outgoing", TimeUnit.DAYS, FieldSpec.DataType.INT)
-      .build();
 
   @Test
   public void testRecordReaders()
@@ -115,15 +102,20 @@ public class RecordReaderSampleDataTest {
     }
   }
 
+  /**
+   * Tests that record extractor is able to handle missing fields correctly (incoming and outgoing are missing from data)
+   * @throws Exception
+   */
   @Test
-  public void testDifferentIncomingOutgoing()
+  public void testRecordExtractorAbsentFields()
       throws Exception {
+    HashSet<String> sourceFields = Sets.newHashSet("incoming", "time_day", "outgoing", "column2");
     try (RecordReader avroRecordReader = RecordReaderFactory
-        .getRecordReader(FileFormat.AVRO, AVRO_SAMPLE_DATA_FILE, Sets.newHashSet("time_day", "column2"), null);
+        .getRecordReader(FileFormat.AVRO, AVRO_SAMPLE_DATA_FILE, sourceFields, null);
         RecordReader csvRecordReader = RecordReaderFactory
-            .getRecordReader(FileFormat.CSV, CSV_SAMPLE_DATA_FILE, Sets.newHashSet("time_day", "column2"), null);
+            .getRecordReader(FileFormat.CSV, CSV_SAMPLE_DATA_FILE, sourceFields, null);
         RecordReader jsonRecordReader = RecordReaderFactory
-            .getRecordReader(FileFormat.JSON, JSON_SAMPLE_DATA_FILE, Sets.newHashSet("time_day", "column2"), null)) {
+            .getRecordReader(FileFormat.JSON, JSON_SAMPLE_DATA_FILE, sourceFields, null)) {
       int numRecords = 0;
       while (avroRecordReader.hasNext()) {
         assertTrue(csvRecordReader.hasNext());
@@ -138,78 +130,9 @@ public class RecordReaderSampleDataTest {
 
         // Check the values from the first record
         if (numRecords == 1) {
-          // Incoming time column
           assertEquals(Long.valueOf(avroRecord.getValue("time_day").toString()), new Long(1072889503L));
-
-          // Outgoing time column
           assertEquals(avroRecord.getValue("column2"), 231355578);
-        }
-      }
-      assertEquals(numRecords, 10001);
-    }
-  }
-
-  @Test
-  public void testNoIncoming()
-      throws Exception {
-    try (RecordReader avroRecordReader = RecordReaderFactory
-        .getRecordReader(FileFormat.AVRO, AVRO_SAMPLE_DATA_FILE, SCHEMA_NO_INCOMING.getColumnNames(), null);
-        RecordReader csvRecordReader = RecordReaderFactory
-            .getRecordReader(FileFormat.CSV, CSV_SAMPLE_DATA_FILE, SCHEMA_NO_INCOMING.getColumnNames(), null);
-        RecordReader jsonRecordReader = RecordReaderFactory
-            .getRecordReader(FileFormat.JSON, JSON_SAMPLE_DATA_FILE, SCHEMA_NO_INCOMING.getColumnNames(), null)) {
-      int numRecords = 0;
-      while (avroRecordReader.hasNext()) {
-        assertTrue(csvRecordReader.hasNext());
-        assertTrue(jsonRecordReader.hasNext());
-        numRecords++;
-
-        GenericRow avroRecord = avroRecordReader.next();
-        GenericRow csvRecord = csvRecordReader.next();
-        GenericRow jsonRecord = jsonRecordReader.next();
-        checkEqualCSV(avroRecord, csvRecord);
-        checkEqual(avroRecord, jsonRecord);
-
-        // Check the values from the first record
-        if (numRecords == 1) {
-          // Incoming time column should be null
           assertNull(avroRecord.getValue("incoming"));
-
-          // Outgoing time column
-          assertEquals(avroRecord.getValue("time_day"), 1072889503);
-        }
-      }
-      assertEquals(numRecords, 10001);
-    }
-  }
-
-  @Test
-  public void testNoOutgoing()
-      throws Exception {
-    try (RecordReader avroRecordReader = RecordReaderFactory
-        .getRecordReader(FileFormat.AVRO, AVRO_SAMPLE_DATA_FILE, Sets.newHashSet("time_day", "outgoing"), null);
-        RecordReader csvRecordReader = RecordReaderFactory
-            .getRecordReader(FileFormat.CSV, CSV_SAMPLE_DATA_FILE, Sets.newHashSet("time_day", "outgoing"), null);
-        RecordReader jsonRecordReader = RecordReaderFactory
-            .getRecordReader(FileFormat.JSON, JSON_SAMPLE_DATA_FILE, Sets.newHashSet("time_day", "outgoing"), null)) {
-      int numRecords = 0;
-      while (avroRecordReader.hasNext()) {
-        assertTrue(csvRecordReader.hasNext());
-        assertTrue(jsonRecordReader.hasNext());
-        numRecords++;
-
-        GenericRow avroRecord = avroRecordReader.next();
-        GenericRow csvRecord = csvRecordReader.next();
-        GenericRow jsonRecord = jsonRecordReader.next();
-        checkEqualCSV(avroRecord, csvRecord);
-        checkEqual(avroRecord, jsonRecord);
-
-        // Check the values from the first record
-        if (numRecords == 1) {
-          // Incoming time column
-          assertEquals(Long.valueOf(avroRecord.getValue("time_day").toString()), new Long(1072889503L));
-
-          // Outgoing time column should be null
           assertNull(avroRecord.getValue("outgoing"));
         }
       }
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/recordtransformer/ExpressionTransformerTimeTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/recordtransformer/ExpressionTransformerTimeTest.java
index aa86b69..1b30046 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/data/recordtransformer/ExpressionTransformerTimeTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/data/recordtransformer/ExpressionTransformerTimeTest.java
@@ -45,7 +45,7 @@ public class ExpressionTransformerTimeTest {
 
     // 1] only incoming defined - doesn't create DefaultTimeSpecEvaluator. Incoming value used as is.
     pinotSchema = new Schema.SchemaBuilder()
-        .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.MILLISECONDS, "incoming")).build();
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.MILLISECONDS, "incoming"), null).build();
 
     // correct value for incoming - use incoming
     expressionTransformer = new ExpressionTransformer(pinotSchema);
@@ -163,7 +163,7 @@ public class ExpressionTransformerTimeTest {
     // 5] SIMPLE_DATE_FORMAT
     // When incoming and outgoing spec are the same, any time format should work
     pinotSchema = new Schema.SchemaBuilder().addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS,
-        TimeGranularitySpec.TimeFormat.SIMPLE_DATE_FORMAT.toString(), "time")).build();
+        TimeGranularitySpec.TimeFormat.SIMPLE_DATE_FORMAT.toString(), "time"), null).build();
     expressionTransformer = new ExpressionTransformer(pinotSchema);
     genericRow = new GenericRow();
     genericRow.putValue("time", 20180101);
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/generator/SegmentGeneratorConfigTest.java b/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/generator/SegmentGeneratorConfigTest.java
index f7c6b3b..e217ba0 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/generator/SegmentGeneratorConfigTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/generator/SegmentGeneratorConfigTest.java
@@ -37,7 +37,7 @@ public class SegmentGeneratorConfigTest {
   @Test
   public void testEpochTime() {
     Schema schema = new Schema.SchemaBuilder()
-        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch")).build();
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
     TableConfig tableConfig =
         new TableConfigBuilder(TableType.OFFLINE).setTableName("test").setTimeColumnName("daysSinceEpoch").build();
     // table config provided
@@ -58,7 +58,7 @@ public class SegmentGeneratorConfigTest {
   @Test
   public void testSimpleDateFormat() {
     Schema schema = new Schema.SchemaBuilder().addTime(new TimeGranularitySpec(FieldSpec.DataType.STRING, TimeUnit.DAYS,
-        TimeGranularitySpec.TimeFormat.SIMPLE_DATE_FORMAT + ":yyyyMMdd", "Date")).build();
+        TimeGranularitySpec.TimeFormat.SIMPLE_DATE_FORMAT + ":yyyyMMdd", "Date"), null).build();
     TableConfig tableConfig =
         new TableConfigBuilder(TableType.OFFLINE).setTableName("test").setTimeColumnName("Date").build();
 
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplAggregateMetricsTest.java b/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplAggregateMetricsTest.java
index e08e6b3..580e284 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplAggregateMetricsTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplAggregateMetricsTest.java
@@ -30,6 +30,7 @@ import org.apache.pinot.spi.data.DimensionFieldSpec;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.MetricFieldSpec;
 import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.TimeGranularitySpec;
 import org.apache.pinot.spi.data.readers.GenericRow;
 import org.apache.pinot.spi.stream.StreamMessageMetadata;
 import org.testng.Assert;
@@ -54,7 +55,8 @@ public class MutableSegmentImplAggregateMetricsTest {
     Schema schema = new Schema.SchemaBuilder().setSchemaName("testSchema")
         .addSingleValueDimension(DIMENSION_1, FieldSpec.DataType.INT)
         .addSingleValueDimension(DIMENSION_2, FieldSpec.DataType.STRING).addMetric(METRIC, FieldSpec.DataType.LONG)
-        .addMetric(METRIC_2, FieldSpec.DataType.FLOAT).addTime(TIME_COLUMN, TimeUnit.DAYS, FieldSpec.DataType.INT)
+        .addMetric(METRIC_2, FieldSpec.DataType.FLOAT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, TIME_COLUMN), null)
         .build();
     // Add virtual columns, which should not be aggregated
     DimensionFieldSpec virtualDimensionFieldSpec =
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/plan/maker/MetadataAndDictionaryAggregationPlanMakerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/plan/maker/MetadataAndDictionaryAggregationPlanMakerTest.java
index 4e006a0..c6bd73d 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/plan/maker/MetadataAndDictionaryAggregationPlanMakerTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/plan/maker/MetadataAndDictionaryAggregationPlanMakerTest.java
@@ -44,6 +44,7 @@ import org.apache.pinot.spi.config.table.TableType;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.FieldSpec.DataType;
 import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.TimeGranularitySpec;
 import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
@@ -82,8 +83,8 @@ public class MetadataAndDictionaryAggregationPlanMakerTest {
         .addSingleValueDimension("column9", FieldSpec.DataType.INT)
         .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
         .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
-        .addMetric("column18", FieldSpec.DataType.INT).addTime("daysSinceEpoch", 1, TimeUnit.DAYS, DataType.INT)
-        .build();
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(DataType.INT, 1, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
     TableConfig tableConfig =
         new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch").build();
 
diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/BaseMultiValueQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/BaseMultiValueQueriesTest.java
index ae332a4..add44b5 100644
--- a/pinot-core/src/test/java/org/apache/pinot/queries/BaseMultiValueQueriesTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/queries/BaseMultiValueQueriesTest.java
@@ -37,6 +37,7 @@ import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
 import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
 import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
 import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.spi.data.TimeGranularitySpec;
 import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
@@ -94,7 +95,8 @@ public abstract class BaseMultiValueQueriesTest extends BaseQueriesTest {
         .addMultiValueDimension("column6", FieldSpec.DataType.INT)
         .addMultiValueDimension("column7", FieldSpec.DataType.INT)
         .addSingleValueDimension("column8", FieldSpec.DataType.INT).addMetric("column9", FieldSpec.DataType.INT)
-        .addMetric("column10", FieldSpec.DataType.INT).addTime("daysSinceEpoch", TimeUnit.DAYS, FieldSpec.DataType.INT)
+        .addMetric("column10", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null)
         .build();
     TableConfig tableConfig =
         new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch").build();
diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/BaseSingleValueQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/BaseSingleValueQueriesTest.java
index d86440e..e46e87f 100644
--- a/pinot-core/src/test/java/org/apache/pinot/queries/BaseSingleValueQueriesTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/queries/BaseSingleValueQueriesTest.java
@@ -38,6 +38,7 @@ import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.config.table.TableType;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.TimeGranularitySpec;
 import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
@@ -98,7 +99,8 @@ public abstract class BaseSingleValueQueriesTest extends BaseQueriesTest {
         .addSingleValueDimension("column9", FieldSpec.DataType.INT)
         .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
         .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
-        .addMetric("column18", FieldSpec.DataType.INT).addTime("daysSinceEpoch", TimeUnit.DAYS, FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null)
         .build();
     TableConfig tableConfig =
         new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch").build();
diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/FastHllQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/FastHllQueriesTest.java
index 50716b9..8b95d9b 100644
--- a/pinot-core/src/test/java/org/apache/pinot/queries/FastHllQueriesTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/queries/FastHllQueriesTest.java
@@ -43,6 +43,7 @@ import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
 import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.TimeGranularitySpec;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -164,7 +165,8 @@ public class FastHllQueriesTest extends BaseQueriesTest {
         .addSingleValueDimension("column9", FieldSpec.DataType.INT)
         .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
         .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
-        .addMetric("column18", FieldSpec.DataType.INT).addTime("daysSinceEpoch", TimeUnit.DAYS, FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null)
         .addSingleValueDimension("column17_HLL", FieldSpec.DataType.STRING)
         .addSingleValueDimension("column18_HLL", FieldSpec.DataType.STRING).build();
 
diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/TransformQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/TransformQueriesTest.java
index e0f08d7..2344fe1 100644
--- a/pinot-core/src/test/java/org/apache/pinot/queries/TransformQueriesTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/queries/TransformQueriesTest.java
@@ -35,6 +35,7 @@ import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.common.response.broker.AggregationResult;
 import org.apache.pinot.common.response.broker.BrokerResponseNative;
 import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.spi.data.TimeGranularitySpec;
 import org.apache.pinot.spi.utils.TimeUtils;
 import org.apache.pinot.spi.data.readers.GenericRow;
 import org.apache.pinot.core.data.manager.SegmentDataManager;
@@ -86,7 +87,8 @@ public class TransformQueriesTest extends BaseQueriesTest {
         new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension(D1, FieldSpec.DataType.STRING)
             .addSingleValueDimension(M1, FieldSpec.DataType.INT).addSingleValueDimension(M2, FieldSpec.DataType.INT)
             .addSingleValueDimension(M3, FieldSpec.DataType.LONG).addSingleValueDimension(M4, FieldSpec.DataType.LONG)
-            .addTime(TIME, TimeUnit.MILLISECONDS, FieldSpec.DataType.LONG).build();
+            .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.MILLISECONDS, TIME), null)
+            .build();
     _tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName("test").setTimeColumnName(TIME).build();
   }
 
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LuceneRealtimeClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LuceneRealtimeClusterIntegrationTest.java
index ba07a9f..19bfcbb 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LuceneRealtimeClusterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LuceneRealtimeClusterIntegrationTest.java
@@ -70,7 +70,7 @@ public class LuceneRealtimeClusterIntegrationTest extends BaseClusterIntegration
     _schema = new Schema.SchemaBuilder().setSchemaName(TABLE_NAME)
         .addSingleValueDimension(SKILLS_TEXT_COL_NAME, FieldSpec.DataType.STRING)
         .addMetric(INT_COL_NAME, FieldSpec.DataType.INT)
-        .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.MILLISECONDS, TIME_COL_NAME))
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.MILLISECONDS, TIME_COL_NAME), null)
         .build();
 
     // Start the Pinot cluster
diff --git a/pinot-plugins/pinot-input-format/pinot-avro-base/src/test/java/org/apache/pinot/plugin/inputformat/avro/AvroRecordToPinotRowGeneratorTest.java b/pinot-plugins/pinot-input-format/pinot-avro-base/src/test/java/org/apache/pinot/plugin/inputformat/avro/AvroRecordToPinotRowGeneratorTest.java
index fc9029c..2108bdc 100644
--- a/pinot-plugins/pinot-input-format/pinot-avro-base/src/test/java/org/apache/pinot/plugin/inputformat/avro/AvroRecordToPinotRowGeneratorTest.java
+++ b/pinot-plugins/pinot-input-format/pinot-avro-base/src/test/java/org/apache/pinot/plugin/inputformat/avro/AvroRecordToPinotRowGeneratorTest.java
@@ -27,6 +27,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericData;
 import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.TimeGranularitySpec;
 import org.apache.pinot.spi.data.readers.GenericRow;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -43,10 +44,6 @@ public class AvroRecordToPinotRowGeneratorTest {
     GenericData.Record avroRecord = new GenericData.Record(avroSchema);
     avroRecord.put("incomingTime", 12345L);
 
-    org.apache.pinot.spi.data.Schema pinotSchema =
-        new org.apache.pinot.spi.data.Schema.SchemaBuilder().setSchemaName("testSchema")
-            .addTime("incomingTime", TimeUnit.MILLISECONDS, FieldSpec.DataType.LONG, "outgoingTime", TimeUnit.DAYS,
-                FieldSpec.DataType.INT).build();
     Set<String> sourceFields = Sets.newHashSet("incomingTime", "outgoingTime");
 
     AvroRecordExtractor avroRecordExtractor = new AvroRecordExtractor();
diff --git a/pinot-plugins/pinot-input-format/pinot-avro-base/src/test/java/org/apache/pinot/plugin/inputformat/avro/AvroUtilsTest.java b/pinot-plugins/pinot-input-format/pinot-avro-base/src/test/java/org/apache/pinot/plugin/inputformat/avro/AvroUtilsTest.java
index 5bff367..7bed45f 100644
--- a/pinot-plugins/pinot-input-format/pinot-avro-base/src/test/java/org/apache/pinot/plugin/inputformat/avro/AvroUtilsTest.java
+++ b/pinot-plugins/pinot-input-format/pinot-avro-base/src/test/java/org/apache/pinot/plugin/inputformat/avro/AvroUtilsTest.java
@@ -26,6 +26,7 @@ import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.FieldSpec.DataType;
 import org.apache.pinot.spi.data.FieldSpec.FieldType;
 import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.TimeGranularitySpec;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -60,7 +61,7 @@ public class AvroUtilsTest {
     Schema expectedSchema = new Schema.SchemaBuilder().addSingleValueDimension("d1", DataType.STRING)
         .addSingleValueDimension("d2", DataType.LONG).addSingleValueDimension("d3", DataType.STRING)
         .addMetric("m1", DataType.INT).addMetric("m2", DataType.INT)
-        .addTime("hoursSinceEpoch", TimeUnit.HOURS, DataType.LONG).build();
+        .addTime(new TimeGranularitySpec(DataType.LONG, TimeUnit.HOURS, "hoursSinceEpoch"), null).build();
     Assert.assertEquals(expectedSchema, inferredPinotSchema);
   }
 }
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFieldSpec.java b/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFieldSpec.java
index 5349f9c..1fe75a4 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFieldSpec.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFieldSpec.java
@@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.google.common.base.Preconditions;
+import javax.annotation.Nullable;
 import org.apache.pinot.spi.utils.EqualityUtils;
 
 
@@ -41,10 +42,7 @@ public final class DateTimeFieldSpec extends FieldSpec {
   }
 
   /**
-   * @param name
-   *
-   * @param dataType
-   *
+   * Constructs a DateTimeFieldSpec with basic fields name, dataType, format and granularity
    * @param format - defines how to interpret the numeric value in the date time column.
    * Format has to follow the pattern - size:timeunit:timeformat, where
    * size and timeUnit together define the granularity of the time column value.
@@ -79,6 +77,16 @@ public final class DateTimeFieldSpec extends FieldSpec {
     _granularity = granularity;
   }
 
+  /**
+   * Constructs a DateTimeFieldSpec with basic fields - name, dataType, format, granularity - and also with defaultNullValue and transformFunction
+   */
+  public DateTimeFieldSpec(String name, DataType dataType, String format, String granularity,
+      @Nullable Object defaultNullValue, @Nullable String transformFunction) {
+    this(name, dataType, format, granularity);
+    setDefaultNullValue(defaultNullValue);
+    setTransformFunction(transformFunction);
+  }
+
   @JsonIgnore
   @Override
   public FieldType getFieldType() {
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/data/FieldSpec.java b/pinot-spi/src/main/java/org/apache/pinot/spi/data/FieldSpec.java
index 068977e..8a6edf2 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/data/FieldSpec.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/data/FieldSpec.java
@@ -243,7 +243,7 @@ public abstract class FieldSpec implements Comparable<FieldSpec> {
   }
 
   // Required by JSON de-serializer. DO NOT REMOVE.
-  public void setTransformFunction(String transformFunction) {
+  public void setTransformFunction(@Nullable String transformFunction) {
     _transformFunction = transformFunction;
   }
 
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/data/Schema.java b/pinot-spi/src/main/java/org/apache/pinot/spi/data/Schema.java
index ad1281f..8fb05b3 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/data/Schema.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/data/Schema.java
@@ -36,6 +36,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
 import org.apache.pinot.spi.data.FieldSpec.DataType;
 import org.apache.pinot.spi.data.FieldSpec.FieldType;
 import org.apache.pinot.spi.utils.EqualityUtils;
@@ -436,118 +437,86 @@ public final class Schema {
       return this;
     }
 
+    /**
+     * Add single value dimensionFieldSpec
+     */
     public SchemaBuilder addSingleValueDimension(String dimensionName, DataType dataType) {
       _schema.addField(new DimensionFieldSpec(dimensionName, dataType, true));
       return this;
     }
 
+    /**
+     * Add single value dimensionFieldSpec with a defaultNullValue
+     */
     public SchemaBuilder addSingleValueDimension(String dimensionName, DataType dataType, Object defaultNullValue) {
       _schema.addField(new DimensionFieldSpec(dimensionName, dataType, true, defaultNullValue));
       return this;
     }
 
+    /**
+     * Add multi value dimensionFieldSpec
+     */
     public SchemaBuilder addMultiValueDimension(String dimensionName, DataType dataType) {
       _schema.addField(new DimensionFieldSpec(dimensionName, dataType, false));
       return this;
     }
 
+    /**
+     * Add single value dimensionFieldSpec with defaultNullValue
+     */
     public SchemaBuilder addMultiValueDimension(String dimensionName, DataType dataType, Object defaultNullValue) {
       _schema.addField(new DimensionFieldSpec(dimensionName, dataType, false, defaultNullValue));
       return this;
     }
 
+    /**
+     * Add metricFieldSpec
+     */
     public SchemaBuilder addMetric(String metricName, DataType dataType) {
       _schema.addField(new MetricFieldSpec(metricName, dataType));
       return this;
     }
 
+    /**
+     * Add metricFieldSpec with defaultNullValue
+     */
     public SchemaBuilder addMetric(String metricName, DataType dataType, Object defaultNullValue) {
       _schema.addField(new MetricFieldSpec(metricName, dataType, defaultNullValue));
       return this;
     }
 
-    public SchemaBuilder addTime(String incomingName, TimeUnit incomingTimeUnit, DataType incomingDataType) {
-      _schema.addField(new TimeFieldSpec(incomingName, incomingDataType, incomingTimeUnit));
-      return this;
-    }
-
-    public SchemaBuilder addTime(String incomingName, TimeUnit incomingTimeUnit, DataType incomingDataType,
-        Object defaultNullValue) {
-      _schema.addField(new TimeFieldSpec(incomingName, incomingDataType, incomingTimeUnit, defaultNullValue));
-      return this;
-    }
-
-    public SchemaBuilder addTime(String incomingName, TimeUnit incomingTimeUnit, DataType incomingDataType,
-        String outgoingName, TimeUnit outgoingTimeUnit, DataType outgoingDataType) {
-      _schema.addField(
-          new TimeFieldSpec(incomingName, incomingDataType, incomingTimeUnit, outgoingName, outgoingDataType,
-              outgoingTimeUnit));
-      return this;
-    }
-
-    public SchemaBuilder addTime(String incomingName, TimeUnit incomingTimeUnit, DataType incomingDataType,
-        String outgoingName, TimeUnit outgoingTimeUnit, DataType outgoingDataType, Object defaultNullValue) {
-      _schema.addField(
-          new TimeFieldSpec(incomingName, incomingDataType, incomingTimeUnit, outgoingName, outgoingDataType,
-              outgoingTimeUnit, defaultNullValue));
-      return this;
-    }
-
-    public SchemaBuilder addTime(String incomingName, int incomingTimeUnitSize, TimeUnit incomingTimeUnit,
-        DataType incomingDataType) {
-      _schema.addField(new TimeFieldSpec(incomingName, incomingDataType, incomingTimeUnitSize, incomingTimeUnit));
-      return this;
-    }
-
-    public SchemaBuilder addTime(String incomingName, int incomingTimeUnitSize, TimeUnit incomingTimeUnit,
-        DataType incomingDataType, Object defaultNullValue) {
-      _schema.addField(
-          new TimeFieldSpec(incomingName, incomingDataType, incomingTimeUnitSize, incomingTimeUnit, defaultNullValue));
-      return this;
-    }
-
-    public SchemaBuilder addTime(String incomingName, int incomingTimeUnitSize, TimeUnit incomingTimeUnit,
-        DataType incomingDataType, String outgoingName, int outgoingTimeUnitSize, TimeUnit outgoingTimeUnit,
-        DataType outgoingDataType) {
-      _schema.addField(
-          new TimeFieldSpec(incomingName, incomingDataType, incomingTimeUnitSize, incomingTimeUnit, outgoingName,
-              outgoingDataType, outgoingTimeUnitSize, outgoingTimeUnit));
-      return this;
-    }
-
-    public SchemaBuilder addTime(String incomingName, int incomingTimeUnitSize, TimeUnit incomingTimeUnit,
-        DataType incomingDataType, String outgoingName, int outgoingTimeUnitSize, TimeUnit outgoingTimeUnit,
-        DataType outgoingDataType, Object defaultNullValue) {
-      _schema.addField(
-          new TimeFieldSpec(incomingName, incomingDataType, incomingTimeUnitSize, incomingTimeUnit, outgoingName,
-              outgoingDataType, outgoingTimeUnitSize, outgoingTimeUnit, defaultNullValue));
-      return this;
-    }
-
-    public SchemaBuilder addTime(TimeGranularitySpec incomingTimeGranularitySpec) {
-      _schema.addField(new TimeFieldSpec(incomingTimeGranularitySpec));
-      return this;
-    }
-
-    public SchemaBuilder addTime(TimeGranularitySpec incomingTimeGranularitySpec, Object defaultNullValue) {
-      _schema.addField(new TimeFieldSpec(incomingTimeGranularitySpec, defaultNullValue));
-      return this;
-    }
-
+    /**
+     * Add timeFieldSpec with incoming and outgoing granularity spec
+     * TODO: This is going to be deprecated in favor of addDateTime().
+     *  Many tests use this to construct Schema with TimeFieldSpec.
+     *  This will continue to exist for a while, as it helps to test backward compatibility of schemas containing TimeFieldSpec
+     */
     public SchemaBuilder addTime(TimeGranularitySpec incomingTimeGranularitySpec,
-        TimeGranularitySpec outgoingTimeGranularitySpec) {
-      _schema.addField(new TimeFieldSpec(incomingTimeGranularitySpec, outgoingTimeGranularitySpec));
+        @Nullable TimeGranularitySpec outgoingTimeGranularitySpec) {
+      if (outgoingTimeGranularitySpec != null) {
+        _schema.addField(new TimeFieldSpec(incomingTimeGranularitySpec, outgoingTimeGranularitySpec));
+      } else {
+        _schema.addField(new TimeFieldSpec(incomingTimeGranularitySpec));
+      }
       return this;
     }
 
-    public SchemaBuilder addTime(TimeGranularitySpec incomingTimeGranularitySpec,
-        TimeGranularitySpec outgoingTimeGranularitySpec, Object defaultNullValue) {
-      _schema.addField(new TimeFieldSpec(incomingTimeGranularitySpec, outgoingTimeGranularitySpec, defaultNullValue));
+    /**
+     * Add dateTimeFieldSpec with basic fields
+     */
+    public SchemaBuilder addDateTime(String name, DataType dataType, String format, String granularity) {
+      _schema.addField(new DateTimeFieldSpec(name, dataType, format, granularity));
       return this;
     }
 
-    public SchemaBuilder addDateTime(String name, DataType dataType, String format, String granularity) {
-      _schema.addField(new DateTimeFieldSpec(name, dataType, format, granularity));
+    /**
+     * Add dateTimeFieldSpec with basic fields plus defaultNullValue and transformFunction
+     */
+    public SchemaBuilder addDateTime(String name, DataType dataType, String format, String granularity,
+        @Nullable Object defaultNullValue, @Nullable String transformFunction) {
+      DateTimeFieldSpec dateTimeFieldSpec =
+          new DateTimeFieldSpec(name, dataType, format, granularity, defaultNullValue, transformFunction);
+      _schema.addField(dateTimeFieldSpec);
       return this;
     }
 
diff --git a/pinot-spi/src/test/java/org/apache/pinot/spi/data/DateTimeFieldSpecUtilsTest.java b/pinot-spi/src/test/java/org/apache/pinot/spi/data/DateTimeFieldSpecUtilsTest.java
index 8bd3cbe..837cd1d 100644
--- a/pinot-spi/src/test/java/org/apache/pinot/spi/data/DateTimeFieldSpecUtilsTest.java
+++ b/pinot-spi/src/test/java/org/apache/pinot/spi/data/DateTimeFieldSpecUtilsTest.java
@@ -75,8 +75,8 @@ public class DateTimeFieldSpecUtilsTest {
     // transform function
     timeFieldSpec = new TimeFieldSpec(new TimeGranularitySpec(DataType.INT, TimeUnit.HOURS, "incoming"));
     timeFieldSpec.setTransformFunction("toEpochHours(timestamp)");
-    expectedDateTimeFieldSpec = new DateTimeFieldSpec("incoming", DataType.INT, "1:HOURS:EPOCH", "1:HOURS");
-    expectedDateTimeFieldSpec.setTransformFunction("toEpochHours(timestamp)");
+    expectedDateTimeFieldSpec =
+        new DateTimeFieldSpec("incoming", DataType.INT, "1:HOURS:EPOCH", "1:HOURS", null, "toEpochHours(timestamp)");
     actualDateTimeFieldSpec = Schema.convertToDateTimeFieldSpec(timeFieldSpec);
     Assert.assertEquals(actualDateTimeFieldSpec, expectedDateTimeFieldSpec);
 
@@ -101,69 +101,72 @@ public class DateTimeFieldSpecUtilsTest {
     // millis to hours
     timeFieldSpec = new TimeFieldSpec(new TimeGranularitySpec(DataType.LONG, TimeUnit.MILLISECONDS, "incoming"),
         new TimeGranularitySpec(DataType.LONG, TimeUnit.HOURS, "outgoing"));
-    expectedDateTimeFieldSpec = new DateTimeFieldSpec("outgoing", DataType.LONG, "1:HOURS:EPOCH", "1:HOURS");
-    expectedDateTimeFieldSpec.setTransformFunction("toEpochHours(incoming)");
+    expectedDateTimeFieldSpec =
+        new DateTimeFieldSpec("outgoing", DataType.LONG, "1:HOURS:EPOCH", "1:HOURS", null, "toEpochHours(incoming)");
     actualDateTimeFieldSpec = Schema.convertToDateTimeFieldSpec(timeFieldSpec);
     Assert.assertEquals(actualDateTimeFieldSpec, expectedDateTimeFieldSpec);
 
     // millis to bucketed minutes
     timeFieldSpec = new TimeFieldSpec(new TimeGranularitySpec(DataType.LONG, TimeUnit.MILLISECONDS, "incoming"),
         new TimeGranularitySpec(DataType.LONG, 10, TimeUnit.MINUTES, "outgoing"));
-    expectedDateTimeFieldSpec = new DateTimeFieldSpec("outgoing", DataType.LONG, "10:MINUTES:EPOCH", "10:MINUTES");
-    expectedDateTimeFieldSpec.setTransformFunction("toEpochMinutesBucket(incoming, 10)");
+    expectedDateTimeFieldSpec = new DateTimeFieldSpec("outgoing", DataType.LONG, "10:MINUTES:EPOCH", "10:MINUTES", null,
+        "toEpochMinutesBucket(incoming, 10)");
     actualDateTimeFieldSpec = Schema.convertToDateTimeFieldSpec(timeFieldSpec);
     Assert.assertEquals(actualDateTimeFieldSpec, expectedDateTimeFieldSpec);
 
     // days to millis
     timeFieldSpec = new TimeFieldSpec(new TimeGranularitySpec(DataType.INT, TimeUnit.DAYS, "incoming"),
         new TimeGranularitySpec(DataType.LONG, TimeUnit.MILLISECONDS, "outgoing"));
-    expectedDateTimeFieldSpec = new DateTimeFieldSpec("outgoing", DataType.LONG, "1:MILLISECONDS:EPOCH", "1:MILLISECONDS");
-    expectedDateTimeFieldSpec.setTransformFunction("fromEpochDays(incoming)");
+    expectedDateTimeFieldSpec =
+        new DateTimeFieldSpec("outgoing", DataType.LONG, "1:MILLISECONDS:EPOCH", "1:MILLISECONDS", null,
+            "fromEpochDays(incoming)");
     actualDateTimeFieldSpec = Schema.convertToDateTimeFieldSpec(timeFieldSpec);
     Assert.assertEquals(actualDateTimeFieldSpec, expectedDateTimeFieldSpec);
 
     // bucketed minutes to millis
     timeFieldSpec = new TimeFieldSpec(new TimeGranularitySpec(DataType.LONG, 5, TimeUnit.MINUTES, "incoming"),
         new TimeGranularitySpec(DataType.LONG, TimeUnit.MILLISECONDS, "outgoing"));
-    expectedDateTimeFieldSpec = new DateTimeFieldSpec("outgoing", DataType.LONG, "1:MILLISECONDS:EPOCH", "1:MILLISECONDS");
-    expectedDateTimeFieldSpec.setTransformFunction("fromEpochMinutesBucket(incoming, 5)");
+    expectedDateTimeFieldSpec =
+        new DateTimeFieldSpec("outgoing", DataType.LONG, "1:MILLISECONDS:EPOCH", "1:MILLISECONDS", null,
+            "fromEpochMinutesBucket(incoming, 5)");
     actualDateTimeFieldSpec = Schema.convertToDateTimeFieldSpec(timeFieldSpec);
     Assert.assertEquals(actualDateTimeFieldSpec, expectedDateTimeFieldSpec);
 
     // hours to days
     timeFieldSpec = new TimeFieldSpec(new TimeGranularitySpec(DataType.INT, TimeUnit.HOURS, "incoming"),
         new TimeGranularitySpec(DataType.INT, TimeUnit.DAYS, "outgoing"));
-    expectedDateTimeFieldSpec = new DateTimeFieldSpec("outgoing", DataType.INT, "1:DAYS:EPOCH", "1:DAYS");
-    expectedDateTimeFieldSpec.setTransformFunction("toEpochDays(fromEpochHours(incoming))");
+    expectedDateTimeFieldSpec = new DateTimeFieldSpec("outgoing", DataType.INT, "1:DAYS:EPOCH", "1:DAYS", null,
+        "toEpochDays(fromEpochHours(incoming))");
     actualDateTimeFieldSpec = Schema.convertToDateTimeFieldSpec(timeFieldSpec);
     Assert.assertEquals(actualDateTimeFieldSpec, expectedDateTimeFieldSpec);
 
     // minutes to hours
     timeFieldSpec = new TimeFieldSpec(new TimeGranularitySpec(DataType.LONG, TimeUnit.MINUTES, "incoming"),
         new TimeGranularitySpec(DataType.LONG, TimeUnit.HOURS, "outgoing"));
-    expectedDateTimeFieldSpec = new DateTimeFieldSpec("outgoing", DataType.LONG, "1:HOURS:EPOCH", "1:HOURS");
-    expectedDateTimeFieldSpec.setTransformFunction("toEpochHours(fromEpochMinutes(incoming))");
+    expectedDateTimeFieldSpec = new DateTimeFieldSpec("outgoing", DataType.LONG, "1:HOURS:EPOCH", "1:HOURS", null,
+        "toEpochHours(fromEpochMinutes(incoming))");
     actualDateTimeFieldSpec = Schema.convertToDateTimeFieldSpec(timeFieldSpec);
     Assert.assertEquals(actualDateTimeFieldSpec, expectedDateTimeFieldSpec);
 
     // bucketed minutes to days
     timeFieldSpec = new TimeFieldSpec(new TimeGranularitySpec(DataType.LONG, 10, TimeUnit.MINUTES, "incoming"),
         new TimeGranularitySpec(DataType.LONG, TimeUnit.DAYS, "outgoing"));
-    expectedDateTimeFieldSpec = new DateTimeFieldSpec("outgoing", DataType.LONG, "1:DAYS:EPOCH", "1:DAYS");
-    expectedDateTimeFieldSpec.setTransformFunction("toEpochDays(fromEpochMinutesBucket(incoming, 10))");
+    expectedDateTimeFieldSpec = new DateTimeFieldSpec("outgoing", DataType.LONG, "1:DAYS:EPOCH", "1:DAYS", null,
+        "toEpochDays(fromEpochMinutesBucket(incoming, 10))");
     actualDateTimeFieldSpec = Schema.convertToDateTimeFieldSpec(timeFieldSpec);
     Assert.assertEquals(actualDateTimeFieldSpec, expectedDateTimeFieldSpec);
 
     // seconds to bucketed minutes
     timeFieldSpec = new TimeFieldSpec(new TimeGranularitySpec(DataType.LONG, TimeUnit.SECONDS, "incoming"),
         new TimeGranularitySpec(DataType.LONG, 5, TimeUnit.MINUTES, "outgoing"));
-    expectedDateTimeFieldSpec = new DateTimeFieldSpec("outgoing", DataType.LONG, "5:MINUTES:EPOCH", "5:MINUTES");
-    expectedDateTimeFieldSpec.setTransformFunction("toEpochMinutesBucket(fromEpochSeconds(incoming), 5)");
+    expectedDateTimeFieldSpec = new DateTimeFieldSpec("outgoing", DataType.LONG, "5:MINUTES:EPOCH", "5:MINUTES", null,
+        "toEpochMinutesBucket(fromEpochSeconds(incoming), 5)");
     actualDateTimeFieldSpec = Schema.convertToDateTimeFieldSpec(timeFieldSpec);
     Assert.assertEquals(actualDateTimeFieldSpec, expectedDateTimeFieldSpec);
 
     // simple date format to millis
-    timeFieldSpec = new TimeFieldSpec(new TimeGranularitySpec(DataType.LONG, TimeUnit.DAYS, "SIMPLE_DATE_FORMAT:yyyyMMdd", "incoming"),
+    timeFieldSpec = new TimeFieldSpec(
+        new TimeGranularitySpec(DataType.LONG, TimeUnit.DAYS, "SIMPLE_DATE_FORMAT:yyyyMMdd", "incoming"),
         new TimeGranularitySpec(DataType.LONG, TimeUnit.MILLISECONDS, "outgoing"));
     try {
       Schema.convertToDateTimeFieldSpec(timeFieldSpec);
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/GenerateDataCommand.java b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/GenerateDataCommand.java
index 4283e6b..e57a01a 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/GenerateDataCommand.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/GenerateDataCommand.java
@@ -25,6 +25,7 @@ import org.apache.pinot.spi.data.FieldSpec.FieldType;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.data.Schema.SchemaBuilder;
 import org.apache.pinot.spi.data.TimeFieldSpec;
+import org.apache.pinot.spi.data.TimeGranularitySpec;
 import org.apache.pinot.spi.data.readers.FileFormat;
 import org.apache.pinot.spi.utils.JsonUtils;
 import org.apache.pinot.tools.Command;
@@ -220,7 +221,7 @@ public class GenerateDataCommand extends AbstractBaseAdminCommand implements Com
     schemaBuilder.addSingleValueDimension("name", DataType.STRING);
     schemaBuilder.addSingleValueDimension("age", DataType.INT);
     schemaBuilder.addMetric("percent", DataType.FLOAT);
-    schemaBuilder.addTime("days", TimeUnit.DAYS, DataType.LONG);
+    schemaBuilder.addTime(new TimeGranularitySpec(DataType.LONG, TimeUnit.DAYS, "days"), null);
 
     Schema schema = schemaBuilder.build();
     System.out.println(JsonUtils.objectToPrettyString(schema));


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