You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by jl...@apache.org on 2019/07/16 18:16:40 UTC

[incubator-pinot] branch master updated: 3891: Check for validity of segment start/end time during segment generation (#4368)

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

jlli 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 7b1715c  3891: Check for validity of segment start/end time during segment generation (#4368)
7b1715c is described below

commit 7b1715c44f1a049d01cf739ab90d9defbda5f667
Author: Sidd <si...@gmail.com>
AuthorDate: Tue Jul 16 11:16:35 2019 -0700

    3891: Check for validity of segment start/end time during segment generation (#4368)
    
    * 3891: Check for validity of segment start/end time during
    segment generation.
    
    This will allow us to fail-fast if there is incorrect data
    as opposed to detecting this later during segment upload.
    The check during segment upload is still retained though
    
    * Introduced a config option in SegmentGeneratorConfig to disable (if needed)
    the time column value validity check in the segment generation code path.
    
    The config option is enabled by default. Also, the tests that were generating
    segments were failing due to the newly introduced check in segment generation code.
    Very few of these tests generate test data themselves and so that was modified
    to generate time column data under acceptable range. But most of them
    were taking data from external avro file. So the tests explicitly
    disable the check for now until a new PR comes in that is going
    to modify the avro files with acceptable time column data.
    
    * Fix PinotOutputFormatTest
    
    * Check for segment time column unit if type is INT
    and convert to millis before doing validation check
    against min-max allowed values in segment generation code
    
    * Fix PinotSegmentUtil to generate valid data for all time units
---
 .../apache/pinot/common/utils/time/TimeUtils.java  |  15 +++
 .../controller/api/upload/SegmentValidator.java    |   2 +-
 .../generator/SegmentGeneratorConfig.java          |  10 ++
 .../apache/pinot/core/minion/SegmentConverter.java |  18 ++-
 .../converter/RealtimeSegmentConverter.java        |   6 +
 .../creator/impl/SegmentColumnarIndexCreator.java  | 135 ++++++++++++++++++++-
 .../pinot/core/data/readers/PinotSegmentUtil.java  |  74 ++++++++---
 .../pinot/core/minion/SegmentConverterTest.java    |  16 ++-
 ...adataAndDictionaryAggregationPlanMakerTest.java |  12 ++
 .../core/segment/index/ColumnMetadataTest.java     |   6 +
 .../segment/index/SegmentMetadataImplTest.java     |   6 +
 .../SegmentV1V2ToV3FormatConverterTest.java        |   6 +
 .../SegmentGenerationWithTimeColumnTest.java       |  36 ++++--
 .../index/loader/SegmentPreProcessorTest.java      |   6 +
 .../startree/StarTreeIndexTestSegmentHelper.java   |   6 +
 .../pinot/core/startree/TestStarTreeMetadata.java  |   6 +
 .../hll/SegmentWithHllIndexCreateHelper.java       |   6 +
 .../org/apache/pinot/core/util/CrcUtilsTest.java   |   6 +
 .../pinot/queries/BaseMultiValueQueriesTest.java   |   6 +
 .../pinot/queries/BaseSingleValueQueriesTest.java  |   6 +
 .../apache/pinot/queries/FastHllQueriesTest.java   |   6 +
 .../segments/v1/creator/DictionariesTest.java      |   6 +
 .../pinot/segments/v1/creator/IntArraysTest.java   |   6 +
 23 files changed, 364 insertions(+), 38 deletions(-)

diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/time/TimeUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/time/TimeUtils.java
index fe88253..9ce6d44 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/utils/time/TimeUtils.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/time/TimeUtils.java
@@ -167,4 +167,19 @@ public class TimeUtils {
     }
     return periodStr;
   }
+
+  /**
+   * Verify that start and end time (should be in milliseconds from epoch) of the segment
+   * are in valid range.
+   * @param startMillis start time (in milliseconds)
+   * @param endMillis end time (in milliseconds)
+   * @return true if start and end time are in range, false otherwise
+   *
+   * Note: this function assumes that given times are in milliseconds. The
+   * caller should take care of converting to millis from epoch before
+   * trying to validate the times.
+   */
+  public static boolean checkSegmentTimeValidity(final long startMillis, final long endMillis) {
+    return timeValueInValidRange(startMillis) && timeValueInValidRange(endMillis);
+  }
 }
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/SegmentValidator.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/SegmentValidator.java
index 39a9657..bea9f74 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/SegmentValidator.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/upload/SegmentValidator.java
@@ -156,7 +156,7 @@ public class SegmentValidator {
     long startMillis = interval.getStartMillis();
     long endMillis = interval.getEndMillis();
 
-    if (!TimeUtils.timeValueInValidRange(startMillis) || !TimeUtils.timeValueInValidRange(endMillis)) {
+    if (!TimeUtils.checkSegmentTimeValidity(startMillis, endMillis)) {
       Date minDate = new Date(TimeUtils.getValidMinTimeMillis());
       Date maxDate = new Date(TimeUtils.getValidMaxTimeMillis());
 
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/generator/SegmentGeneratorConfig.java b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/generator/SegmentGeneratorConfig.java
index 4b4251c..ae6c95d 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/generator/SegmentGeneratorConfig.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/generator/SegmentGeneratorConfig.java
@@ -112,6 +112,7 @@ public class SegmentGeneratorConfig {
   private String _simpleDateFormat = null;
   // Use on-heap or off-heap memory to generate index (currently only affect inverted index and star-tree v2)
   private boolean _onHeap = false;
+  private boolean _checkTimeColumnValidityDuringGeneration = true;
 
   public SegmentGeneratorConfig() {
   }
@@ -160,6 +161,7 @@ public class SegmentGeneratorConfig {
     _simpleDateFormat = config._simpleDateFormat;
     _onHeap = config._onHeap;
     _recordReaderPath = config._recordReaderPath;
+    _checkTimeColumnValidityDuringGeneration = config._checkTimeColumnValidityDuringGeneration;
   }
 
   /**
@@ -593,6 +595,14 @@ public class SegmentGeneratorConfig {
     _onHeap = onHeap;
   }
 
+  public boolean isCheckTimeColumnValidityDuringGeneration() {
+    return _checkTimeColumnValidityDuringGeneration;
+  }
+
+  public void setCheckTimeColumnValidityDuringGeneration(boolean checkTimeColumnValidityDuringGeneration) {
+    _checkTimeColumnValidityDuringGeneration = checkTimeColumnValidityDuringGeneration;
+  }
+
   public Map<String, ChunkCompressorFactory.CompressionType> getRawIndexCompressionType() {
     return _rawIndexCompressionType;
   }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/minion/SegmentConverter.java b/pinot-core/src/main/java/org/apache/pinot/core/minion/SegmentConverter.java
index 933b4db..1723104 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/minion/SegmentConverter.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/minion/SegmentConverter.java
@@ -69,11 +69,13 @@ public class SegmentConverter {
   private RecordAggregator _recordAggregator;
   private List<String> _groupByColumns;
   private IndexingConfig _indexingConfig;
+  private boolean _checkTimeValidityDuringGeneration;
 
   public SegmentConverter(@Nonnull List<File> inputIndexDirs, @Nonnull File workingDir, @Nonnull String tableName,
       @Nonnull String segmentName, int totalNumPartition, @Nonnull RecordTransformer recordTransformer,
       @Nullable RecordPartitioner recordPartitioner, @Nullable RecordAggregator recordAggregator,
-      @Nullable List<String> groupByColumns, @Nullable IndexingConfig indexingConfig) {
+      @Nullable List<String> groupByColumns, @Nullable IndexingConfig indexingConfig,
+      boolean checkTimeValidityDuringGeneration) {
     _inputIndexDirs = inputIndexDirs;
     _workingDir = workingDir;
     _recordTransformer = recordTransformer;
@@ -86,6 +88,8 @@ public class SegmentConverter {
     _recordAggregator = recordAggregator;
     _groupByColumns = groupByColumns;
     _indexingConfig = indexingConfig;
+
+    _checkTimeValidityDuringGeneration = checkTimeValidityDuringGeneration;
   }
 
   public List<File> convertSegment()
@@ -148,6 +152,7 @@ public class SegmentConverter {
     segmentGeneratorConfig.setOutDir(outputPath);
     segmentGeneratorConfig.setTableName(tableName);
     segmentGeneratorConfig.setSegmentName(segmentName);
+    segmentGeneratorConfig.setCheckTimeColumnValidityDuringGeneration(_checkTimeValidityDuringGeneration);
     if (indexingConfig != null) {
       segmentGeneratorConfig.setInvertedIndexCreationColumns(indexingConfig.getInvertedIndexColumns());
       if (indexingConfig.getStarTreeIndexSpec() != null) {
@@ -174,6 +179,9 @@ public class SegmentConverter {
     private List<String> _groupByColumns;
     private IndexingConfig _indexingConfig;
 
+    // enabled by default
+    private boolean _checkTimeValidityDuringGeneration = true;
+
     public Builder setInputIndexDirs(List<File> inputIndexDirs) {
       _inputIndexDirs = inputIndexDirs;
       return this;
@@ -224,6 +232,11 @@ public class SegmentConverter {
       return this;
     }
 
+    public Builder setCheckTimeValidityDuringGeneration(final boolean checkTimeValidity) {
+      _checkTimeValidityDuringGeneration = checkTimeValidity;
+      return this;
+    }
+
     public SegmentConverter build() {
       // Check that the group-by columns and record aggregator are configured together
       if (_groupByColumns != null && _groupByColumns.size() > 0) {
@@ -235,7 +248,8 @@ public class SegmentConverter {
       }
 
       return new SegmentConverter(_inputIndexDirs, _workingDir, _tableName, _segmentName, _totalNumPartition,
-          _recordTransformer, _recordPartitioner, _recordAggregator, _groupByColumns, _indexingConfig);
+          _recordTransformer, _recordPartitioner, _recordAggregator, _groupByColumns, _indexingConfig,
+          _checkTimeValidityDuringGeneration);
     }
   }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/realtime/converter/RealtimeSegmentConverter.java b/pinot-core/src/main/java/org/apache/pinot/core/realtime/converter/RealtimeSegmentConverter.java
index d833603..a9c34a9 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/realtime/converter/RealtimeSegmentConverter.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/realtime/converter/RealtimeSegmentConverter.java
@@ -95,6 +95,12 @@ public class RealtimeSegmentConverter {
       reader = new RealtimeSegmentRecordReader(realtimeSegmentImpl, dataSchema, sortedColumn);
     }
     SegmentGeneratorConfig genConfig = new SegmentGeneratorConfig(dataSchema);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. We don't want the realtime consumption to stop (if an exception
+    // is thrown) and thus the time validity check is explicitly disabled for
+    // realtime segment generation
+    genConfig.setCheckTimeColumnValidityDuringGeneration(false);
     if (invertedIndexColumns != null && !invertedIndexColumns.isEmpty()) {
       for (String column : invertedIndexColumns) {
         genConfig.createInvertedIndexForColumn(column);
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/SegmentColumnarIndexCreator.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/SegmentColumnarIndexCreator.java
index 0898cab..c497c61 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/SegmentColumnarIndexCreator.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/SegmentColumnarIndexCreator.java
@@ -23,6 +23,7 @@ import com.google.common.collect.Iterables;
 import java.io.File;
 import java.io.IOException;
 import java.util.Collection;
+import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -36,6 +37,7 @@ import org.apache.pinot.common.data.FieldSpec.FieldType;
 import org.apache.pinot.common.data.Schema;
 import org.apache.pinot.common.data.StarTreeIndexSpec;
 import org.apache.pinot.common.utils.FileUtils;
+import org.apache.pinot.common.utils.time.TimeUtils;
 import org.apache.pinot.core.data.GenericRow;
 import org.apache.pinot.core.data.partition.PartitionFunction;
 import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
@@ -131,8 +133,8 @@ public class SegmentColumnarIndexCreator implements SegmentCreator {
 
         // Initialize dictionary creator
         SegmentDictionaryCreator dictionaryCreator =
-            new SegmentDictionaryCreator(indexCreationInfo.getSortedUniqueElementsArray(),
-                fieldSpec, _indexDir, indexCreationInfo.isUseVarLengthDictionary());
+            new SegmentDictionaryCreator(indexCreationInfo.getSortedUniqueElementsArray(), fieldSpec, _indexDir,
+                indexCreationInfo.isUseVarLengthDictionary());
         _dictionaryCreatorMap.put(columnName, dictionaryCreator);
 
         // Create dictionary
@@ -340,6 +342,7 @@ public class SegmentColumnarIndexCreator implements SegmentCreator {
     if (timeColumnIndexCreationInfo != null) {
       // Use start/end time in config if defined
       if (config.getStartTime() != null) {
+        checkTime(config, config.getStartTime(), config.getEndTime(), segmentName);
         properties.setProperty(SEGMENT_START_TIME, config.getStartTime());
         properties.setProperty(SEGMENT_END_TIME, Preconditions.checkNotNull(config.getEndTime()));
         properties.setProperty(TIME_UNIT, Preconditions.checkNotNull(config.getSegmentTimeUnit()));
@@ -348,12 +351,17 @@ public class SegmentColumnarIndexCreator implements SegmentCreator {
         Object maxTime = Preconditions.checkNotNull(timeColumnIndexCreationInfo.getMax());
 
         if (config.getTimeColumnType() == SegmentGeneratorConfig.TimeColumnType.SIMPLE_DATE) {
-          // For simple date format, convert time value into millis since epoch
+          // For TimeColumnType.SIMPLE_DATE_FORMAT, convert time value into millis since epoch
           DateTimeFormatter dateTimeFormatter = DateTimeFormat.forPattern(config.getSimpleDateFormat());
-          properties.setProperty(SEGMENT_START_TIME, dateTimeFormatter.parseMillis(minTime.toString()));
-          properties.setProperty(SEGMENT_END_TIME, dateTimeFormatter.parseMillis(maxTime.toString()));
+          final long minTimeMillis = dateTimeFormatter.parseMillis(minTime.toString());
+          final long maxTimeMillis = dateTimeFormatter.parseMillis(maxTime.toString());
+          checkTime(config, minTimeMillis, maxTimeMillis, segmentName);
+          properties.setProperty(SEGMENT_START_TIME, minTimeMillis);
+          properties.setProperty(SEGMENT_END_TIME, maxTimeMillis);
           properties.setProperty(TIME_UNIT, TimeUnit.MILLISECONDS);
         } else {
+          // by default, time column type is TimeColumnType.EPOCH
+          checkTime(config, minTime, maxTime, segmentName);
           properties.setProperty(SEGMENT_START_TIME, minTime);
           properties.setProperty(SEGMENT_END_TIME, maxTime);
           properties.setProperty(TIME_UNIT, Preconditions.checkNotNull(config.getSegmentTimeUnit()));
@@ -393,6 +401,123 @@ public class SegmentColumnarIndexCreator implements SegmentCreator {
     properties.save();
   }
 
+  /**
+   * Check for the validity of segment start and end time
+   * @param startTime segment start time
+   * @param endTime segment end time
+   * @param segmentName segment name
+   */
+  private void checkTime(final SegmentGeneratorConfig config, final Object startTime, final Object endTime,
+      final String segmentName) {
+    if (!config.isCheckTimeColumnValidityDuringGeneration()) {
+      return;
+    }
+
+    if (startTime == null || endTime == null) {
+      throw new RuntimeException("Expecting non-null start/end time for segment: " + segmentName);
+    }
+
+    if (!(startTime.getClass().equals(endTime.getClass()))) {
+      final StringBuilder err = new StringBuilder();
+      err.append("Start and end time of segment should be of same type.").append(" segment name: ").append(segmentName)
+          .append(" start time: ").append(startTime).append(" end time: ").append(endTime).append(" start time class: ")
+          .append(startTime.getClass()).append(" end time class: ").append(endTime.getClass());
+      throw new RuntimeException(err.toString());
+    }
+
+    long start;
+    long end;
+
+    final String cl = startTime.getClass().getSimpleName();
+
+    switch (cl) {
+      case "Long":
+        start = (long) startTime;
+        end = (long) endTime;
+        break;
+      case "String":
+        start = Long.parseLong((String) startTime);
+        end = Long.parseLong((String) endTime);
+        break;
+      case "Integer":
+        start = ((Integer) startTime).longValue();
+        end = ((Integer) endTime).longValue();
+        break;
+      default:
+        final StringBuilder err = new StringBuilder();
+        err.append("Unable to interpret type of time column value. Failed to validate start and end time of segment")
+            .append(" uninterpreted type: ").append(startTime.getClass()).append(" start time: ").append(startTime)
+            .append(" end time: ").append(endTime).append(" time column name: ").append(config.getTimeColumnName())
+            .append(" segment name: ").append(segmentName).append(" segment time column unit: ")
+            .append(config.getSegmentTimeUnit().toString()).append(" segment time column type: ")
+            .append(config.getTimeColumnType().toString()).append(" time field spec data type: ")
+            .append(config.getSchema().getTimeFieldSpec().getDataType().toString());
+        LOGGER.error(err.toString());
+        throw new RuntimeException(err.toString());
+    }
+
+    // note that handling of SimpleDateFormat (TimeColumnType.SIMPLE)
+    // is done by the caller of this function that converts the simple format
+    // into millis since epoch before calling this function for validation.
+    // For TimeColumnType.EPOCH, the time field spec could still have unit
+    // as any of the following and we need to convert to millis for doing the
+    // min-max comparison against TimeUtils.getValidMinTimeMillis() and
+    // TimeUtils.getValidMaxTimeMillis()
+    if (config.getTimeColumnType() == SegmentGeneratorConfig.TimeColumnType.EPOCH) {
+      switch (config.getSegmentTimeUnit()) {
+        case DAYS:
+          start = TimeUnit.DAYS.toMillis(start);
+          end = TimeUnit.DAYS.toMillis(end);
+          break;
+        case HOURS:
+          start = TimeUnit.HOURS.toMillis(start);
+          end = TimeUnit.HOURS.toMillis(end);
+          break;
+        case MINUTES:
+          start = TimeUnit.MINUTES.toMillis(start);
+          end = TimeUnit.MINUTES.toMillis(end);
+          break;
+        case SECONDS:
+          start = TimeUnit.SECONDS.toMillis(start);
+          end = TimeUnit.SECONDS.toMillis(end);
+          break;
+        case MICROSECONDS:
+          start = TimeUnit.MICROSECONDS.toMillis(start);
+          end = TimeUnit.MICROSECONDS.toMillis(end);
+          break;
+        case NANOSECONDS:
+          start = TimeUnit.NANOSECONDS.toMillis(start);
+          end = TimeUnit.NANOSECONDS.toMillis(end);
+          break;
+        default:
+          if (config.getSegmentTimeUnit() != TimeUnit.MILLISECONDS) {
+            // we should never be here
+            final StringBuilder err = new StringBuilder();
+            err.append("Unexpected time unit: ").append(config.getSegmentTimeUnit().toString())
+                .append(" for time column: ").append(config.getTimeColumnName()).append(" for segment: ")
+                .append(segmentName);
+            LOGGER.error(err.toString());
+            throw new RuntimeException(err.toString());
+          }
+      }
+    }
+
+    if (!TimeUtils.checkSegmentTimeValidity(start, end)) {
+      final Date minDate = new Date(TimeUtils.getValidMinTimeMillis());
+      final Date maxDate = new Date(TimeUtils.getValidMaxTimeMillis());
+      final StringBuilder err = new StringBuilder();
+      err.append("Invalid start/end time.").append(" segment name: ").append(segmentName).append(" time column name: ")
+          .append(config.getTimeColumnName()).append(" given start time: ").append(start).append("ms")
+          .append(" given end time: ").append(end).append("ms").append(" start and end time must be between ")
+          .append(minDate).append(" and ").append(maxDate).append(" segment time column unit: ")
+          .append(config.getSegmentTimeUnit().toString()).append(" segment time column type: ")
+          .append(config.getTimeColumnType().toString()).append(" time field spec data type: ")
+          .append(config.getSchema().getTimeFieldSpec().getDataType().toString());
+      LOGGER.error(err.toString());
+      throw new RuntimeException(err.toString());
+    }
+  }
+
   public static void addColumnMetadataInfo(PropertiesConfiguration properties, String column,
       ColumnIndexCreationInfo columnIndexCreationInfo, int totalDocs, int totalRawDocs, int totalAggDocs,
       FieldSpec fieldSpec, boolean hasDictionary, int dictionaryElementSize, boolean hasInvertedIndex,
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/readers/PinotSegmentUtil.java b/pinot-core/src/test/java/org/apache/pinot/core/data/readers/PinotSegmentUtil.java
index 90fc992..fe311ef 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/data/readers/PinotSegmentUtil.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/data/readers/PinotSegmentUtil.java
@@ -18,6 +18,7 @@
  */
 package org.apache.pinot.core.data.readers;
 
+import com.google.common.base.Preconditions;
 import java.io.File;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -27,9 +28,13 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.pinot.common.data.FieldSpec;
 import org.apache.pinot.common.data.Schema;
+import org.apache.pinot.common.data.TimeFieldSpec;
+import org.apache.pinot.common.utils.time.TimeUtils;
 import org.apache.pinot.core.data.GenericRow;
 import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
 import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
@@ -47,16 +52,16 @@ public class PinotSegmentUtil {
 
   public static List<GenericRow> createTestData(Schema schema, int numRows) {
     List<GenericRow> rows = new ArrayList<>();
-    final Random random = new Random();
+    final ThreadLocalRandom random = ThreadLocalRandom.current();
     Map<String, Object> fields;
     for (int i = 0; i < numRows; i++) {
       fields = new HashMap<>();
       for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
         Object value;
         if (fieldSpec.isSingleValueField()) {
-          value = generateSingleValue(random, fieldSpec.getDataType());
+          value = generateSingleValue(random, fieldSpec);
         } else {
-          value = generateMultiValue(random, fieldSpec.getDataType());
+          value = generateMultiValue(random, fieldSpec);
         }
         fields.put(fieldSpec.getName(), value);
       }
@@ -94,27 +99,58 @@ public class PinotSegmentUtil {
     return segmentIndexDir;
   }
 
-  private static Object generateSingleValue(Random random, FieldSpec.DataType dataType) {
-    switch (dataType) {
-      case INT:
-        return Math.abs(random.nextInt());
-      case LONG:
-        return Math.abs(random.nextLong());
-      case FLOAT:
-        return Math.abs(random.nextFloat());
-      case DOUBLE:
-        return Math.abs(random.nextDouble());
-      case STRING:
-        return RandomStringUtils.randomAlphabetic(DEFAULT_STRING_VALUE_LENGTH);
-      default:
-        throw new IllegalStateException("Illegal data type");
+  private static Object generateSingleValue(ThreadLocalRandom random, FieldSpec fieldSpec) {
+    if (fieldSpec instanceof TimeFieldSpec) {
+      // explicitly generate the time column values within allowed range so that
+      // segment generation code doesn't throw exception
+      TimeFieldSpec timeFieldSpec = (TimeFieldSpec)fieldSpec;
+      TimeUnit unit = timeFieldSpec.getIncomingGranularitySpec().getTimeType();
+      final long milliMin = TimeUtils.getValidMinTimeMillis();
+      final long milliMax = TimeUtils.getValidMaxTimeMillis();
+      final long daysMin = TimeUnit.DAYS.convert(milliMin, TimeUnit.MILLISECONDS);
+      final long daysMax = TimeUnit.DAYS.convert(milliMax, TimeUnit.MILLISECONDS);
+      final long hoursMin = TimeUnit.HOURS.convert(milliMin, TimeUnit.MILLISECONDS);
+      final long hoursMax = TimeUnit.HOURS.convert(milliMax, TimeUnit.MILLISECONDS);
+      final long minutesMin = TimeUnit.MINUTES.convert(milliMin, TimeUnit.MILLISECONDS);
+      final long minutesMax = TimeUnit.MINUTES.convert(milliMax, TimeUnit.MILLISECONDS);
+      switch (unit) {
+        case MILLISECONDS:
+          return random.nextLong(milliMin, milliMax);
+        case SECONDS:
+          return random.nextLong(milliMin/1000, milliMax/1000);
+        case MICROSECONDS:
+          return random.nextLong(milliMin*1000, milliMax*1000);
+        case NANOSECONDS:
+          return random.nextLong(milliMin*1000*1000, milliMax*1000*1000);
+        case DAYS:
+          return random.nextLong(daysMin, daysMax);
+        case HOURS:
+          return random.nextLong(hoursMin, hoursMax);
+        case MINUTES:
+          return random.nextLong(minutesMin, minutesMax);
+      }
+    } else {
+      switch (fieldSpec.getDataType()) {
+        case INT:
+          return Math.abs(random.nextInt());
+        case LONG:
+          return Math.abs(random.nextLong());
+        case FLOAT:
+          return Math.abs(random.nextFloat());
+        case DOUBLE:
+          return Math.abs(random.nextDouble());
+        case STRING:
+          return RandomStringUtils.randomAlphabetic(DEFAULT_STRING_VALUE_LENGTH);
+      }
     }
+
+    throw new IllegalStateException("Illegal data type");
   }
 
-  private static Object[] generateMultiValue(Random random, FieldSpec.DataType dataType) {
+  private static Object[] generateMultiValue(ThreadLocalRandom random, FieldSpec fieldSpec) {
     Object[] value = new Object[DEFAULT_NUM_MULTIVALUE];
     for (int i = 0; i < DEFAULT_NUM_MULTIVALUE; i++) {
-      value[i] = generateSingleValue(random, dataType);
+      value[i] = generateSingleValue(random, fieldSpec);
     }
     return value;
   }
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/minion/SegmentConverterTest.java b/pinot-core/src/test/java/org/apache/pinot/core/minion/SegmentConverterTest.java
index bffef66..e10e980 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/minion/SegmentConverterTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/minion/SegmentConverterTest.java
@@ -29,6 +29,7 @@ import org.apache.pinot.common.data.FieldSpec;
 import org.apache.pinot.common.data.MetricFieldSpec;
 import org.apache.pinot.common.data.Schema;
 import org.apache.pinot.common.data.TimeFieldSpec;
+import org.apache.pinot.common.utils.time.TimeUtils;
 import org.apache.pinot.core.data.GenericRow;
 import org.apache.pinot.core.data.readers.GenericRowRecordReader;
 import org.apache.pinot.core.data.readers.PinotSegmentRecordReader;
@@ -58,7 +59,7 @@ public class SegmentConverterTest {
   private static final String T = "t";
 
   private List<File> _segmentIndexDirList;
-  private final long _referenceTimestamp = System.currentTimeMillis();
+  private final long _referenceTimestamp = TimeUtils.getValidMinTimeMillis();
 
   @BeforeClass
   public void setUp()
@@ -142,9 +143,20 @@ public class SegmentConverterTest {
     final BaseDateTimeTransformer dateTimeTransformer =
         DateTimeTransformerFactory.getDateTimeTransformer("1:MILLISECONDS:EPOCH", "1:DAYS:EPOCH", "1:DAYS");
 
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we have explicitly disabled the check since the segment
+    // conversion is happening along with transforming each input record by converting
+    // the time column value (from millis since epoch) into days since epoch. While the
+    // source data is under range, the transformed data (days since epoch) goes out of range
+    // since segment conversion follows the same schema -- it does not create new schema. This
+    // means that time column spec doesn't change and still carries the time unit as milliseconds
+    // for the converted segment even though values we are writing are "days since epoch" and
+    // not "millis since epoch". Thus SegmentColumnarIndexCreator throws exception.
     SegmentConverter segmentConverter =
         new SegmentConverter.Builder().setTableName(TABLE_NAME).setSegmentName("segmentRollupWithTimeConversion")
-            .setInputIndexDirs(_segmentIndexDirList).setWorkingDir(WORKING_DIR).setRecordTransformer((row) -> {
+            .setInputIndexDirs(_segmentIndexDirList).setWorkingDir(WORKING_DIR).setCheckTimeValidityDuringGeneration(false)
+            .setRecordTransformer((row) -> {
           long[] input = new long[1];
           long[] output = new long[1];
           input[0] = (Long) row.getValue(T);
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 19c207b..b6ba799 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
@@ -94,6 +94,12 @@ public class MetadataAndDictionaryAggregationPlanMakerTest {
     segmentGeneratorConfig.setTableName("testTable");
     segmentGeneratorConfig.setSegmentName(SEGMENT_NAME);
     segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setCheckTimeColumnValidityDuringGeneration(false);
     segmentGeneratorConfig
         .setInvertedIndexCreationColumns(Arrays.asList("column6", "column7", "column11", "column17", "column18"));
 
@@ -121,6 +127,12 @@ public class MetadataAndDictionaryAggregationPlanMakerTest {
     segmentGeneratorConfig.setSegmentName(SEGMENT_NAME_STARTREE);
     segmentGeneratorConfig.setOutDir(INDEX_DIR_STARTREE.getAbsolutePath());
     segmentGeneratorConfig.enableStarTreeIndex(new StarTreeIndexSpec());
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setCheckTimeColumnValidityDuringGeneration(false);
 
     // Build the index segment.
     driver = new SegmentIndexCreationDriverImpl();
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/ColumnMetadataTest.java b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/ColumnMetadataTest.java
index c81573a..54886fc 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/ColumnMetadataTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/ColumnMetadataTest.java
@@ -68,6 +68,12 @@ public class ColumnMetadataTest {
             "testTable");
     config.setSegmentNamePostfix("1");
     config.setTimeColumnName("daysSinceEpoch");
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    config.setCheckTimeColumnValidityDuringGeneration(false);
     return config;
   }
 
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/SegmentMetadataImplTest.java b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/SegmentMetadataImplTest.java
index 8557367..30656e3 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/SegmentMetadataImplTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/SegmentMetadataImplTest.java
@@ -57,6 +57,12 @@ public class SegmentMetadataImplTest {
             "testTable");
     config.setSegmentNamePostfix("1");
     config.setTimeColumnName("daysSinceEpoch");
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    config.setCheckTimeColumnValidityDuringGeneration(false);
     final SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
     driver.init(config);
     driver.build();
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/converter/SegmentV1V2ToV3FormatConverterTest.java b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/converter/SegmentV1V2ToV3FormatConverterTest.java
index ece96ea..08d4008 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/converter/SegmentV1V2ToV3FormatConverterTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/converter/SegmentV1V2ToV3FormatConverterTest.java
@@ -64,6 +64,12 @@ public class SegmentV1V2ToV3FormatConverterTest {
             "testTable");
     config.setSegmentNamePostfix("1");
     config.setTimeColumnName("daysSinceEpoch");
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    config.setCheckTimeColumnValidityDuringGeneration(false);
     final SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
     driver.init(config);
     driver.build();
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/creator/SegmentGenerationWithTimeColumnTest.java b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/creator/SegmentGenerationWithTimeColumnTest.java
index 09e9ac2..7f92dbf 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/creator/SegmentGenerationWithTimeColumnTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/creator/SegmentGenerationWithTimeColumnTest.java
@@ -30,6 +30,7 @@ import org.apache.pinot.common.data.DimensionFieldSpec;
 import org.apache.pinot.common.data.FieldSpec;
 import org.apache.pinot.common.data.Schema;
 import org.apache.pinot.common.data.TimeFieldSpec;
+import org.apache.pinot.common.utils.time.TimeUtils;
 import org.apache.pinot.core.data.GenericRow;
 import org.apache.pinot.core.data.readers.GenericRowRecordReader;
 import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
@@ -37,6 +38,7 @@ import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl
 import org.apache.pinot.core.segment.index.SegmentMetadataImpl;
 import org.apache.pinot.core.segment.store.SegmentDirectory;
 import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
 import org.joda.time.LocalDateTime;
 import org.joda.time.format.DateTimeFormat;
 import org.joda.time.format.DateTimeFormatter;
@@ -56,6 +58,7 @@ public class SegmentGenerationWithTimeColumnTest {
 
   private Random _random = new Random(System.nanoTime());
 
+  private long validMinTime = TimeUtils.getValidMinTimeMillis();
   private long minTime;
   private long maxTime;
   private long startTime = System.currentTimeMillis();
@@ -71,7 +74,7 @@ public class SegmentGenerationWithTimeColumnTest {
   public void testSimpleDateSegmentGeneration()
       throws Exception {
     Schema schema = createSchema(true);
-    File segmentDir = buildSegment(schema, true);
+    File segmentDir = buildSegment(schema, true, false);
     SegmentMetadataImpl metadata = SegmentDirectory.loadSegmentMetadata(segmentDir);
     Assert.assertEquals(metadata.getStartTime(), sdfToMillis(minTime));
     Assert.assertEquals(metadata.getEndTime(), sdfToMillis(maxTime));
@@ -81,12 +84,23 @@ public class SegmentGenerationWithTimeColumnTest {
   public void testEpochDateSegmentGeneration()
       throws Exception {
     Schema schema = createSchema(false);
-    File segmentDir = buildSegment(schema, false);
+    File segmentDir = buildSegment(schema, false, false);
     SegmentMetadataImpl metadata = SegmentDirectory.loadSegmentMetadata(segmentDir);
     Assert.assertEquals(metadata.getStartTime(), minTime);
     Assert.assertEquals(metadata.getEndTime(), maxTime);
   }
 
+  @Test
+  public void testSegmentGenerationWithInvalidTime() {
+    Schema schema = createSchema(false);
+    try {
+      buildSegment(schema, false, true);
+      Assert.fail("Expecting exception from buildSegment for invalid start/end time of segment");
+    } catch (Exception e) {
+      Assert.assertTrue(e.getMessage().contains("Invalid start/end time. segment name: testSegment time column name: date"));
+    }
+  }
+
   private Schema createSchema(boolean isSimpleDate) {
     Schema schema = new Schema();
     schema.addField(new DimensionFieldSpec(STRING_COL_NAME, FieldSpec.DataType.STRING, true));
@@ -98,7 +112,8 @@ public class SegmentGenerationWithTimeColumnTest {
     return schema;
   }
 
-  private File buildSegment(Schema schema, boolean isSimpleDate)
+  private File buildSegment(final Schema schema, final boolean isSimpleDate,
+      final boolean isInvalidDate)
       throws Exception {
     SegmentGeneratorConfig config = new SegmentGeneratorConfig(schema);
     config.setRawIndexCreationColumns(schema.getDimensionNames());
@@ -117,7 +132,7 @@ public class SegmentGenerationWithTimeColumnTest {
       for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
         Object value;
 
-        value = getRandomValueForColumn(fieldSpec, isSimpleDate);
+        value = getRandomValueForColumn(fieldSpec, isSimpleDate, isInvalidDate);
         map.put(fieldSpec.getName(), value);
       }
 
@@ -133,18 +148,21 @@ public class SegmentGenerationWithTimeColumnTest {
     return driver.getOutputDirectory();
   }
 
-  private Object getRandomValueForColumn(FieldSpec fieldSpec, boolean isSimpleDate) {
+  private Object getRandomValueForColumn(FieldSpec fieldSpec, boolean isSimpleDate, boolean isInvalidDate) {
     if (fieldSpec.getName().equals(TIME_COL_NAME)) {
-      return getRandomValueForTimeColumn(isSimpleDate);
+      return getRandomValueForTimeColumn(isSimpleDate, isInvalidDate);
     }
     return RawIndexCreatorTest.getRandomValue(_random, fieldSpec.getDataType());
   }
 
-  private Object getRandomValueForTimeColumn(boolean isSimpleDate) {
-    long randomMs = ThreadLocalRandom.current().nextLong(startTime);
+  private Object getRandomValueForTimeColumn(boolean isSimpleDate, boolean isInvalidDate) {
+    long randomMs = ThreadLocalRandom.current().nextLong(validMinTime, startTime);
     long dateColVal = randomMs;
     Object result;
-    if (isSimpleDate) {
+    if (isInvalidDate) {
+      result = new Long(new DateTime(2072, 1, 1, 0, 0, 0, 0, DateTimeZone.UTC).getMillis());
+      return result;
+    } else if (isSimpleDate) {
       DateTime dateTime = new DateTime(randomMs);
       LocalDateTime localDateTime = dateTime.toLocalDateTime();
       int year = localDateTime.getYear();
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/loader/SegmentPreProcessorTest.java b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/loader/SegmentPreProcessorTest.java
index 42cc0de..ea2293a 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/loader/SegmentPreProcessorTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/loader/SegmentPreProcessorTest.java
@@ -119,6 +119,12 @@ public class SegmentPreProcessorTest {
     SegmentGeneratorConfig segmentGeneratorConfig =
         SegmentTestUtils.getSegmentGeneratorConfigWithSchema(_avroFile, INDEX_DIR, "testTable", _schema);
     segmentGeneratorConfig.setInvertedIndexCreationColumns(Collections.singletonList(COLUMN7_NAME));
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setCheckTimeColumnValidityDuringGeneration(false);
     SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
     driver.init(segmentGeneratorConfig);
     driver.build();
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/startree/StarTreeIndexTestSegmentHelper.java b/pinot-core/src/test/java/org/apache/pinot/core/startree/StarTreeIndexTestSegmentHelper.java
index a183a0b..2e57398 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/startree/StarTreeIndexTestSegmentHelper.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/startree/StarTreeIndexTestSegmentHelper.java
@@ -85,6 +85,12 @@ public class StarTreeIndexTestSegmentHelper {
     config.setFormat(FileFormat.AVRO);
     config.setSegmentName(segmentName);
     config.setHllConfig(hllConfig);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    config.setCheckTimeColumnValidityDuringGeneration(false);
 
     List<GenericRow> rows = new ArrayList<>(numRows);
     for (int rowId = 0; rowId < numRows; rowId++) {
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/startree/TestStarTreeMetadata.java b/pinot-core/src/test/java/org/apache/pinot/core/startree/TestStarTreeMetadata.java
index a252822..08ee7b7 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/startree/TestStarTreeMetadata.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/startree/TestStarTreeMetadata.java
@@ -96,6 +96,12 @@ public class TestStarTreeMetadata {
     starTreeIndexSpec.setSkipMaterializationForDimensions(SKIP_MATERIALIZATION_DIMENSIONS);
 
     config.enableStarTreeIndex(starTreeIndexSpec);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    config.setCheckTimeColumnValidityDuringGeneration(false);
 
     final SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
     driver.init(config);
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/startree/hll/SegmentWithHllIndexCreateHelper.java b/pinot-core/src/test/java/org/apache/pinot/core/startree/hll/SegmentWithHllIndexCreateHelper.java
index ca390cb..e36334d 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/startree/hll/SegmentWithHllIndexCreateHelper.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/startree/hll/SegmentWithHllIndexCreateHelper.java
@@ -134,6 +134,12 @@ public class SegmentWithHllIndexCreateHelper {
     segmentGenConfig.createInvertedIndexForAllColumns();
     segmentGenConfig.setSegmentName(segmentName);
     segmentGenConfig.setSegmentNamePostfix("1");
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGenConfig.setCheckTimeColumnValidityDuringGeneration(false);
 
     if (enableStarTree) {
       setupStarTreeConfig(segmentGenConfig);
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/util/CrcUtilsTest.java b/pinot-core/src/test/java/org/apache/pinot/core/util/CrcUtilsTest.java
index f2a794e..af8a36a 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/util/CrcUtilsTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/util/CrcUtilsTest.java
@@ -83,6 +83,12 @@ public class CrcUtilsTest {
             "testTable");
     config.setSegmentNamePostfix("1");
     config.setTimeColumnName("daysSinceEpoch");
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    config.setCheckTimeColumnValidityDuringGeneration(false);
     final SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
     driver.init(config);
     driver.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 d642210..7820861 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
@@ -100,6 +100,12 @@ public abstract class BaseMultiValueQueriesTest extends BaseQueriesTest {
     segmentGeneratorConfig.setTableName("testTable");
     segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
     segmentGeneratorConfig.setInvertedIndexCreationColumns(Arrays.asList("column3", "column7", "column8", "column9"));
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setCheckTimeColumnValidityDuringGeneration(false);
 
     // Build the index segment.
     SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
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 b5b2176..860d61a 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
@@ -101,6 +101,12 @@ public abstract class BaseSingleValueQueriesTest extends BaseQueriesTest {
     segmentGeneratorConfig.setInputFilePath(filePath);
     segmentGeneratorConfig.setTableName("testTable");
     segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setCheckTimeColumnValidityDuringGeneration(false);
     segmentGeneratorConfig
         .setInvertedIndexCreationColumns(Arrays.asList("column6", "column7", "column11", "column17", "column18"));
 
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 dbf0b9b..218ed07 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
@@ -235,6 +235,12 @@ public class FastHllQueriesTest extends BaseQueriesTest {
     segmentGeneratorConfig.setInputFilePath(filePath);
     segmentGeneratorConfig.setTableName("testTable");
     segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setCheckTimeColumnValidityDuringGeneration(false);
     segmentGeneratorConfig
         .setInvertedIndexCreationColumns(Arrays.asList("column6", "column7", "column11", "column17", "column18"));
     if (hasPreGeneratedHllColumns) {
diff --git a/pinot-core/src/test/java/org/apache/pinot/segments/v1/creator/DictionariesTest.java b/pinot-core/src/test/java/org/apache/pinot/segments/v1/creator/DictionariesTest.java
index 468b922..aefce15 100644
--- a/pinot-core/src/test/java/org/apache/pinot/segments/v1/creator/DictionariesTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/segments/v1/creator/DictionariesTest.java
@@ -94,6 +94,12 @@ public class DictionariesTest {
         .getSegmentGenSpecWithSchemAndProjectedColumns(new File(filePath), INDEX_DIR, "time_day", TimeUnit.DAYS,
             "test");
 
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    config.setCheckTimeColumnValidityDuringGeneration(false);
     final SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
     driver.init(config);
     driver.build();
diff --git a/pinot-core/src/test/java/org/apache/pinot/segments/v1/creator/IntArraysTest.java b/pinot-core/src/test/java/org/apache/pinot/segments/v1/creator/IntArraysTest.java
index 29b4244..b4ccc2d 100644
--- a/pinot-core/src/test/java/org/apache/pinot/segments/v1/creator/IntArraysTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/segments/v1/creator/IntArraysTest.java
@@ -70,6 +70,12 @@ public class IntArraysTest {
         .getSegmentGenSpecWithSchemAndProjectedColumns(new File(filePath), INDEX_DIR, "weeksSinceEpochSunday",
             TimeUnit.DAYS, "test");
     config.setTimeColumnName("weeksSinceEpochSunday");
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    config.setCheckTimeColumnValidityDuringGeneration(false);
     driver.init(config);
     driver.build();
 


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