You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by xi...@apache.org on 2022/09/09 08:12:38 UTC

[pinot] branch master updated: Allow skipTimeValueCheck from table config (#9349)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new c8a114db48 Allow skipTimeValueCheck from table config (#9349)
c8a114db48 is described below

commit c8a114db4835ce88654708c26a535f9779dfab15
Author: Xiang Fu <xi...@gmail.com>
AuthorDate: Fri Sep 9 01:12:31 2022 -0700

    Allow skipTimeValueCheck from table config (#9349)
---
 .../PinotSegmentUploadDownloadRestletResource.java |  4 ++-
 ...adataAndDictionaryAggregationPlanMakerTest.java | 18 +++++++----
 .../org/apache/pinot/core/util/CrcUtilsTest.java   |  6 ----
 .../pinot/queries/BaseMultiValueQueriesTest.java   | 18 +++++++----
 .../queries/BaseMultiValueRawQueriesTest.java      | 17 ++++++----
 .../pinot/queries/BaseSingleValueQueriesTest.java  | 18 +++++++----
 .../apache/pinot/queries/FastHllQueriesTest.java   | 19 +++++++----
 .../converter/RealtimeSegmentConverter.java        |  2 +-
 .../recordtransformer/DataTypeTransformer.java     | 14 +++++---
 .../mutable/IntermediateSegmentTest.java           | 13 ++++++--
 .../recordtransformer/RecordTransformerTest.java   | 14 +++++---
 .../local/segment/creator/DictionariesTest.java    |  7 ----
 .../segment/creator/DictionaryOptimiserTest.java   | 20 ++++++------
 .../local/segment/creator/IntArraysTest.java       |  6 ----
 .../local/segment/creator/SegmentTestUtils.java    | 13 +++++++-
 .../local/segment/index/ColumnMetadataTest.java    |  6 ----
 .../segment/index/SegmentMetadataImplTest.java     |  6 ----
 .../SegmentV1V2ToV3FormatConverterTest.java        |  6 ----
 .../index/loader/SegmentPreProcessorTest.java      | 18 +++++++----
 .../BaseDefaultColumnHandlerTest.java              |  6 ----
 .../spi/creator/SegmentGeneratorConfig.java        | 37 +++++++++++++++-------
 .../pinot/spi/config/table/IndexingConfig.java     | 18 -----------
 .../config/table/ingestion/IngestionConfig.java    | 33 +++++++++++++++++++
 .../spi/utils/builder/TableConfigBuilder.java      | 14 --------
 24 files changed, 181 insertions(+), 152 deletions(-)

diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java
index 4fb4e298bf..0d493ab3d0 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java
@@ -326,7 +326,9 @@ public class PinotSegmentUploadDownloadRestletResource {
         throw new ControllerApplicationException(LOGGER, "Failed to find table: " + tableNameWithType,
             Response.Status.BAD_REQUEST);
       }
-      SegmentValidationUtils.validateTimeInterval(segmentMetadata, tableConfig);
+      if (tableConfig.getIngestionConfig() == null || tableConfig.getIngestionConfig().isSegmentTimeValueCheck()) {
+        SegmentValidationUtils.validateTimeInterval(segmentMetadata, tableConfig);
+      }
       if (uploadType != FileUploadDownloadClient.FileUploadType.METADATA) {
         SegmentValidationUtils.checkStorageQuota(tempSegmentDir, segmentMetadata, tableConfig,
             _pinotHelixResourceManager, _controllerConf, _controllerMetrics, _connectionManager, _executor,
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 e49b90e657..4d92fd2841 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
@@ -46,6 +46,7 @@ import org.apache.pinot.segment.spi.index.mutable.ThreadSafeMutableRoaringBitmap
 import org.apache.pinot.spi.config.table.HashFunction;
 import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.ingestion.IngestionConfig;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.FieldSpec.DataType;
 import org.apache.pinot.spi.data.Schema;
@@ -94,8 +95,17 @@ public class MetadataAndDictionaryAggregationPlanMakerTest {
         .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
         .addMetric("column18", FieldSpec.DataType.INT)
         .addTime(new TimeGranularitySpec(DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+    // 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
+    IngestionConfig ingestionConfig = new IngestionConfig();
+    ingestionConfig.setSegmentTimeValueCheck(false);
+    ingestionConfig.setRowTimeValueCheck(false);
     TableConfig tableConfig =
-        new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch").build();
+        new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+            .setIngestionConfig(ingestionConfig).build();
 
     // Create the segment generator config.
     SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(tableConfig, schema);
@@ -103,12 +113,6 @@ 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.setSkipTimeValueCheck(true);
     segmentGeneratorConfig.setInvertedIndexCreationColumns(
         Arrays.asList("column6", "column7", "column11", "column17", "column18"));
 
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 e4cf1bdb3f..0589ecfceb 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
@@ -80,12 +80,6 @@ public class CrcUtilsTest {
         .getSegmentGenSpecWithSchemAndProjectedColumns(new File(filePath), INDEX_DIR, "daysSinceEpoch", TimeUnit.DAYS,
             "testTable");
     config.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
-    config.setSkipTimeValueCheck(true);
     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 927f59e012..50f7211cdc 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
@@ -32,6 +32,7 @@ import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
 import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
 import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.ingestion.IngestionConfig;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.data.TimeGranularitySpec;
@@ -98,8 +99,17 @@ public abstract class BaseMultiValueQueriesTest extends BaseQueriesTest {
         .addSingleValueDimension("column8", FieldSpec.DataType.INT).addMetric("column9", FieldSpec.DataType.INT)
         .addMetric("column10", FieldSpec.DataType.INT)
         .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+    // 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
+    IngestionConfig ingestionConfig = new IngestionConfig();
+    ingestionConfig.setSegmentTimeValueCheck(false);
+    ingestionConfig.setRowTimeValueCheck(false);
     TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setNoDictionaryColumns(Arrays.asList("column5"))
-        .setTableName("testTable").setTimeColumnName("daysSinceEpoch").build();
+        .setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+        .setIngestionConfig(ingestionConfig).build();
 
     // Create the segment generator config.
     SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(tableConfig, schema);
@@ -107,12 +117,6 @@ 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.setSkipTimeValueCheck(true);
 
     // Build the index segment.
     SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/BaseMultiValueRawQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/BaseMultiValueRawQueriesTest.java
index 46dfe2335d..9c08b83ebe 100644
--- a/pinot-core/src/test/java/org/apache/pinot/queries/BaseMultiValueRawQueriesTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/queries/BaseMultiValueRawQueriesTest.java
@@ -32,6 +32,7 @@ import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
 import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
 import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.ingestion.IngestionConfig;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.data.TimeGranularitySpec;
@@ -98,9 +99,17 @@ public class BaseMultiValueRawQueriesTest extends BaseQueriesTest {
         .addSingleValueDimension("column8", FieldSpec.DataType.INT).addMetric("column9", FieldSpec.DataType.INT)
         .addMetric("column10", FieldSpec.DataType.INT)
         .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+    // 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
+    IngestionConfig ingestionConfig = new IngestionConfig();
+    ingestionConfig.setSegmentTimeValueCheck(false);
+    ingestionConfig.setRowTimeValueCheck(false);
     TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable")
         .setTimeColumnName("daysSinceEpoch").setNoDictionaryColumns(Arrays.asList("column5", "column6", "column7"))
-        .build();
+        .setIngestionConfig(ingestionConfig).build();
 
     // Create the segment generator config.
     SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(tableConfig, schema);
@@ -109,12 +118,6 @@ public class BaseMultiValueRawQueriesTest extends BaseQueriesTest {
     segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
     segmentGeneratorConfig.setInvertedIndexCreationColumns(Arrays.asList("column3", "column8", "column9"));
     segmentGeneratorConfig.setRawIndexCreationColumns(Arrays.asList("column5", "column6", "column7"));
-    // 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.setSkipTimeValueCheck(true);
 
     // 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 684c7e0999..cc0b190c2e 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
@@ -32,6 +32,7 @@ import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
 import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
 import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.ingestion.IngestionConfig;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.data.TimeGranularitySpec;
@@ -100,20 +101,23 @@ public abstract class BaseSingleValueQueriesTest extends BaseQueriesTest {
         .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
         .addMetric("column18", FieldSpec.DataType.INT)
         .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+    // 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
+    IngestionConfig ingestionConfig = new IngestionConfig();
+    ingestionConfig.setSegmentTimeValueCheck(false);
+    ingestionConfig.setRowTimeValueCheck(false);
     TableConfig tableConfig =
-        new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch").build();
+        new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+            .setIngestionConfig(ingestionConfig).build();
 
     // Create the segment generator config.
     SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(tableConfig, schema);
     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.setSkipTimeValueCheck(true);
     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 3ab2269969..9d4095f39a 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
@@ -40,6 +40,7 @@ import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
 import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
 import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.ingestion.IngestionConfig;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.data.TimeGranularitySpec;
@@ -174,20 +175,24 @@ public class FastHllQueriesTest extends BaseQueriesTest {
         .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null)
         .addSingleValueDimension("column17_HLL", FieldSpec.DataType.STRING)
         .addSingleValueDimension("column18_HLL", FieldSpec.DataType.STRING).build();
+
+    // 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
+    IngestionConfig ingestionConfig = new IngestionConfig();
+    ingestionConfig.setSegmentTimeValueCheck(false);
+    ingestionConfig.setRowTimeValueCheck(false);
     TableConfig tableConfig =
-        new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch").build();
+        new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+            .setIngestionConfig(ingestionConfig).build();
 
     // Create the segment generator config
     SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(tableConfig, schema);
     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.setSkipTimeValueCheck(true);
     segmentGeneratorConfig.setInvertedIndexCreationColumns(
         Arrays.asList("column6", "column7", "column11", "column17", "column18"));
 
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverter.java
index 2ca1758651..9616a2f529 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverter.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverter.java
@@ -87,7 +87,7 @@ public class RealtimeSegmentConverter {
     // 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.setSkipTimeValueCheck(true);
+    genConfig.setSegmentTimeValueCheck(false);
     if (_invertedIndexColumns != null && !_invertedIndexColumns.isEmpty()) {
       for (String column : _invertedIndexColumns) {
         genConfig.createInvertedIndexForColumn(column);
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/DataTypeTransformer.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/DataTypeTransformer.java
index 14f68f1394..6d21e2514c 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/DataTypeTransformer.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/DataTypeTransformer.java
@@ -51,7 +51,7 @@ public class DataTypeTransformer implements RecordTransformer {
 
   private final Map<String, PinotDataType> _dataTypes = new HashMap<>();
   private final boolean _continueOnError;
-  private final boolean _validateTimeValues;
+  private final boolean _rowTimeValueCheck;
   private final String _timeColumnName;
   private final DateTimeFormatSpec _timeFormatSpec;
 
@@ -61,9 +61,13 @@ public class DataTypeTransformer implements RecordTransformer {
         _dataTypes.put(fieldSpec.getName(), PinotDataType.getPinotDataTypeForIngestion(fieldSpec));
       }
     }
-
-    _continueOnError = tableConfig.getIndexingConfig().isContinueOnError();
-    _validateTimeValues = tableConfig.getIndexingConfig().isValidateTimeValue();
+    if (tableConfig.getIngestionConfig() != null) {
+      _continueOnError = tableConfig.getIngestionConfig().isContinueOnError();
+      _rowTimeValueCheck = tableConfig.getIngestionConfig().isRowTimeValueCheck();
+    } else {
+      _continueOnError = false;
+      _rowTimeValueCheck = false;
+    }
     _timeColumnName = tableConfig.getValidationConfig().getTimeColumnName();
 
     DateTimeFormatSpec timeColumnSpec = null;
@@ -86,7 +90,7 @@ public class DataTypeTransformer implements RecordTransformer {
           continue;
         }
 
-        if (_validateTimeValues && _timeFormatSpec != null && column.equals(_timeColumnName)) {
+        if (_rowTimeValueCheck && _timeFormatSpec != null && column.equals(_timeColumnName)) {
           long timeInMs = _timeFormatSpec.fromFormatToMillis(value.toString());
           if (!TimeUtils.timeValueInValidRange(timeInMs)) {
             if (_continueOnError) {
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/IntermediateSegmentTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/IntermediateSegmentTest.java
index acf7887574..babb8fa1a8 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/IntermediateSegmentTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/IntermediateSegmentTest.java
@@ -43,6 +43,7 @@ import org.apache.pinot.spi.config.table.ColumnPartitionConfig;
 import org.apache.pinot.spi.config.table.SegmentPartitionConfig;
 import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.ingestion.IngestionConfig;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.data.TimeGranularitySpec;
@@ -97,7 +98,6 @@ public class IntermediateSegmentTest {
     segmentGeneratorConfig.setTableName("testTable");
     segmentGeneratorConfig.setSegmentName(SEGMENT_NAME);
     segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
-    segmentGeneratorConfig.setSkipTimeValueCheck(true);
     segmentGeneratorConfig.setInvertedIndexCreationColumns(Arrays.asList("column6", "column7"));
 
     IndexSegment segmentFromIntermediateSegment = buildSegmentFromIntermediateSegment(segmentGeneratorConfig);
@@ -229,10 +229,19 @@ public class IntermediateSegmentTest {
   private static TableConfig createTableConfig(String inputFile) {
     TableConfig tableConfig;
     if (AVRO_DATA_SV.equals(inputFile)) {
+      // 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
+      IngestionConfig ingestionConfig = new IngestionConfig();
+      ingestionConfig.setSegmentTimeValueCheck(false);
+      ingestionConfig.setRowTimeValueCheck(false);
       tableConfig =
           new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
               .setInvertedIndexColumns(Arrays.asList("column6", "column7", "column11", "column17", "column18"))
-              .setSegmentPartitionConfig(getSegmentPartitionConfig()).build();
+              .setSegmentPartitionConfig(getSegmentPartitionConfig())
+              .setIngestionConfig(ingestionConfig).build();
     } else {
       tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").build();
     }
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/recordtransformer/RecordTransformerTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/recordtransformer/RecordTransformerTest.java
index 3100165228..bcf0a40b7e 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/recordtransformer/RecordTransformerTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/recordtransformer/RecordTransformerTest.java
@@ -172,8 +172,10 @@ public class RecordTransformerTest {
       assertThrows(() -> transformer.transform(record));
     }
 
+    IngestionConfig ingestionConfig = new IngestionConfig();
+    ingestionConfig.setContinueOnError(true);
     TableConfig tableConfig =
-        new TableConfigBuilder(TableType.OFFLINE).setContinueOnError(true).setTableName("testTable").build();
+        new TableConfigBuilder(TableType.OFFLINE).setIngestionConfig(ingestionConfig).setTableName("testTable").build();
 
     RecordTransformer transformerWithDefaultNulls = new DataTypeTransformer(tableConfig, schema);
     GenericRow record1 = getRecord();
@@ -203,9 +205,11 @@ public class RecordTransformerTest {
     }
 
     // Invalid Timestamp and Validation enabled
+    IngestionConfig ingestionConfig = new IngestionConfig();
+    ingestionConfig.setRowTimeValueCheck(true);
     tableConfig =
         new TableConfigBuilder(TableType.OFFLINE).setTimeColumnName(timeCol)
-            .setValidateTimeValue(true)
+            .setIngestionConfig(ingestionConfig)
             .setTableName("testTable").build();
 
     RecordTransformer transformerWithValidation = new DataTypeTransformer(tableConfig, schema);
@@ -216,10 +220,12 @@ public class RecordTransformerTest {
     }
 
     // Invalid timestamp, validation enabled and ignoreErrors enabled
+    ingestionConfig = new IngestionConfig();
+    ingestionConfig.setRowTimeValueCheck(true);
+    ingestionConfig.setContinueOnError(true);
     tableConfig =
         new TableConfigBuilder(TableType.OFFLINE).setTimeColumnName(timeCol)
-            .setValidateTimeValue(true)
-            .setContinueOnError(true)
+            .setIngestionConfig(ingestionConfig)
             .setTableName("testTable").build();
 
     transformer = new DataTypeTransformer(tableConfig, schema);
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java
index 893f1db85e..69bc85551c 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java
@@ -96,13 +96,6 @@ public class DictionariesTest {
         SegmentTestUtils.getSegmentGenSpecWithSchemAndProjectedColumns(new File(filePath), INDEX_DIR, "time_day",
             TimeUnit.DAYS, "test");
     _tableConfig = config.getTableConfig();
-
-    // 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.setSkipTimeValueCheck(true);
     final SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
     driver.init(config);
     driver.build();
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionaryOptimiserTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionaryOptimiserTest.java
index 5e2e932080..22450f4d71 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionaryOptimiserTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionaryOptimiserTest.java
@@ -35,6 +35,7 @@ import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
 import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
 import org.apache.pinot.segment.spi.creator.SegmentVersion;
 import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.ingestion.IngestionConfig;
 import org.apache.pinot.spi.data.DimensionFieldSpec;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.MetricFieldSpec;
@@ -76,13 +77,6 @@ public class DictionaryOptimiserTest {
     final SegmentGeneratorConfig config =
         getSegmentGenSpecWithSchemAndProjectedColumns(new File(filePath), INDEX_DIR, "time_column", 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.setSkipTimeValueCheck(true);
     final SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
     driver.init(config);
     driver.build();
@@ -126,9 +120,17 @@ public class DictionaryOptimiserTest {
   public static SegmentGeneratorConfig getSegmentGenSpecWithSchemAndProjectedColumns(File inputAvro, File outputDir,
       String timeColumn, TimeUnit timeUnit, String tableName)
       throws IOException {
-
+    // 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
+    IngestionConfig ingestionConfig = new IngestionConfig();
+    ingestionConfig.setRowTimeValueCheck(false);
+    ingestionConfig.setSegmentTimeValueCheck(false);
     final SegmentGeneratorConfig segmentGenSpec =
-        new SegmentGeneratorConfig(new TableConfigBuilder(TableType.OFFLINE).setTableName(tableName).build(),
+        new SegmentGeneratorConfig(new TableConfigBuilder(TableType.OFFLINE).setTableName(tableName)
+            .setIngestionConfig(ingestionConfig).build(),
             extractSchemaFromAvroWithoutTime(inputAvro));
     segmentGenSpec.setInputFilePath(inputAvro.getAbsolutePath());
     segmentGenSpec.setTimeColumnName(timeColumn);
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/IntArraysTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/IntArraysTest.java
index 3b40f62473..a3d1b82652 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/IntArraysTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/IntArraysTest.java
@@ -67,12 +67,6 @@ public class IntArraysTest {
     final SegmentGeneratorConfig config = SegmentTestUtils
         .getSegmentGenSpecWithSchemAndProjectedColumns(new File(filePath), INDEX_DIR, "weeksSinceEpochSunday",
             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.setSkipTimeValueCheck(true);
     driver.init(config);
     driver.build();
 
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/SegmentTestUtils.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/SegmentTestUtils.java
index d05e056d38..e43aaa0979 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/SegmentTestUtils.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/SegmentTestUtils.java
@@ -39,6 +39,7 @@ import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
 import org.apache.pinot.segment.spi.creator.SegmentVersion;
 import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.ingestion.IngestionConfig;
 import org.apache.pinot.spi.data.DimensionFieldSpec;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.FieldSpec.DataType;
@@ -72,8 +73,18 @@ public class SegmentTestUtils {
   public static SegmentGeneratorConfig getSegmentGenSpecWithSchemAndProjectedColumns(File inputAvro, File outputDir,
       String timeColumn, TimeUnit timeUnit, String tableName)
       throws IOException {
+    // 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
+    IngestionConfig ingestionConfig = new IngestionConfig();
+    ingestionConfig.setRowTimeValueCheck(false);
+    ingestionConfig.setSegmentTimeValueCheck(false);
     final SegmentGeneratorConfig segmentGenSpec =
-        new SegmentGeneratorConfig(new TableConfigBuilder(TableType.OFFLINE).setTableName(tableName).build(),
+        new SegmentGeneratorConfig(
+            new TableConfigBuilder(TableType.OFFLINE).setTableName(tableName).setIngestionConfig(ingestionConfig)
+                .build(),
             extractSchemaFromAvroWithoutTime(inputAvro));
     segmentGenSpec.setInputFilePath(inputAvro.getAbsolutePath());
     segmentGenSpec.setTimeColumnName(timeColumn);
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/ColumnMetadataTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/ColumnMetadataTest.java
index dca96df884..053bc8dc55 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/ColumnMetadataTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/ColumnMetadataTest.java
@@ -72,12 +72,6 @@ public class ColumnMetadataTest {
         .getSegmentGenSpecWithSchemAndProjectedColumns(new File(filePath), INDEX_DIR, "daysSinceEpoch", TimeUnit.HOURS,
             "testTable");
     config.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
-    config.setSkipTimeValueCheck(true);
     return config;
   }
 
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/SegmentMetadataImplTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/SegmentMetadataImplTest.java
index a925350aad..447476acc4 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/SegmentMetadataImplTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/SegmentMetadataImplTest.java
@@ -55,12 +55,6 @@ public class SegmentMetadataImplTest {
         .getSegmentGenSpecWithSchemAndProjectedColumns(new File(filePath), INDEX_DIR, "daysSinceEpoch", TimeUnit.HOURS,
             "testTable");
     config.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
-    config.setSkipTimeValueCheck(true);
     config.setCustomProperties(ImmutableMap.of("custom.k1", "v1", "custom.k2", "v2"));
     final SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
     driver.init(config);
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/converter/SegmentV1V2ToV3FormatConverterTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/converter/SegmentV1V2ToV3FormatConverterTest.java
index 4b4fc1b35d..37149f3278 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/converter/SegmentV1V2ToV3FormatConverterTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/converter/SegmentV1V2ToV3FormatConverterTest.java
@@ -63,12 +63,6 @@ public class SegmentV1V2ToV3FormatConverterTest {
         SegmentTestUtils.getSegmentGenSpecWithSchemAndProjectedColumns(new File(filePath), _indexDir, "daysSinceEpoch",
             TimeUnit.HOURS, "testTable");
     config.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
-    config.setSkipTimeValueCheck(true);
     final SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
     driver.init(config);
     driver.build();
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java
index 832f1c7466..b4de2f5e22 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java
@@ -142,8 +142,18 @@ public class SegmentPreProcessorTest {
     _indexLoadingConfig = new IndexLoadingConfig();
     _indexLoadingConfig.setInvertedIndexColumns(
         new HashSet<>(Arrays.asList(COLUMN1_NAME, COLUMN7_NAME, COLUMN13_NAME, NO_SUCH_COLUMN_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
+    IngestionConfig ingestionConfig = new IngestionConfig();
+    ingestionConfig.setRowTimeValueCheck(false);
+    ingestionConfig.setSegmentTimeValueCheck(false);
     _tableConfig =
-        new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch").build();
+        new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+            .setIngestionConfig(ingestionConfig).build();
     _indexLoadingConfig.setTableConfig(_tableConfig);
 
     ClassLoader classLoader = getClass().getClassLoader();
@@ -191,12 +201,6 @@ public class SegmentPreProcessorTest {
         SegmentTestUtils.getSegmentGeneratorConfigWithSchema(_avroFile, INDEX_DIR, "testTable", _tableConfig, _schema);
     segmentGeneratorConfig.setInvertedIndexCreationColumns(Collections.singletonList(COLUMN7_NAME));
     segmentGeneratorConfig.setRawIndexCreationColumns(Collections.singletonList(EXISTING_STRING_COL_RAW));
-    // 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.setSkipTimeValueCheck(true);
     SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
     driver.init(segmentGeneratorConfig);
     driver.build();
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandlerTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandlerTest.java
index 2fa4effcd1..beee1f8b73 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandlerTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandlerTest.java
@@ -65,12 +65,6 @@ public class BaseDefaultColumnHandlerTest {
             "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.setSkipTimeValueCheck(true);
     final SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
     driver.init(config);
     driver.build();
diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java
index 9dbfbd2fa2..ba765f5755 100644
--- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java
+++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java
@@ -47,6 +47,7 @@ import org.apache.pinot.spi.config.table.SegmentZKPropsConfig;
 import org.apache.pinot.spi.config.table.StarTreeIndexConfig;
 import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.config.table.TimestampIndexGranularity;
+import org.apache.pinot.spi.config.table.ingestion.IngestionConfig;
 import org.apache.pinot.spi.data.DateTimeFieldSpec;
 import org.apache.pinot.spi.data.DateTimeFormatSpec;
 import org.apache.pinot.spi.data.FieldSpec;
@@ -111,10 +112,10 @@ public class SegmentGeneratorConfig implements Serializable {
   private DateTimeFormatSpec _dateTimeFormatSpec = 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 _skipTimeValueCheck = false;
   private boolean _nullHandlingEnabled = false;
   private boolean _continueOnError = false;
-  private boolean _validateTimeValue = false;
+  private boolean _rowTimeValueCheck = true;
+  private boolean _segmentTimeValueCheck = true;
   private boolean _failOnEmptySegment = false;
   private boolean _optimizeDictionaryForMetrics = false;
   private double _noDictionarySizeRatioThreshold = DEFAULT_NO_DICTIONARY_SIZE_RATIO_THRESHOLD;
@@ -222,14 +223,18 @@ public class SegmentGeneratorConfig implements Serializable {
       extractH3IndexConfigsFromTableConfig(tableConfig);
       extractCompressionCodecConfigsFromTableConfig(tableConfig);
 
-      _fstTypeForFSTIndex = tableConfig.getIndexingConfig().getFSTIndexType();
-
+      _fstTypeForFSTIndex = indexingConfig.getFSTIndexType();
       _nullHandlingEnabled = indexingConfig.isNullHandlingEnabled();
-      _continueOnError = indexingConfig.isContinueOnError();
-      _validateTimeValue = indexingConfig.isValidateTimeValue();
       _optimizeDictionaryForMetrics = indexingConfig.isOptimizeDictionaryForMetrics();
       _noDictionarySizeRatioThreshold = indexingConfig.getNoDictionarySizeRatioThreshold();
     }
+
+    IngestionConfig ingestionConfig = tableConfig.getIngestionConfig();
+    if (ingestionConfig != null) {
+      _continueOnError = ingestionConfig.isContinueOnError();
+      _rowTimeValueCheck = ingestionConfig.isRowTimeValueCheck();
+      _segmentTimeValueCheck = ingestionConfig.isSegmentTimeValueCheck();
+    }
   }
 
   public static Schema updateSchemaWithTimestampIndexes(Schema schema,
@@ -722,11 +727,11 @@ public class SegmentGeneratorConfig implements Serializable {
   }
 
   public boolean isSkipTimeValueCheck() {
-    return _skipTimeValueCheck;
+    return !_segmentTimeValueCheck;
   }
 
   public void setSkipTimeValueCheck(boolean skipTimeValueCheck) {
-    _skipTimeValueCheck = skipTimeValueCheck;
+    _segmentTimeValueCheck = !skipTimeValueCheck;
   }
 
   public Map<String, ChunkCompressionType> getRawIndexCompressionType() {
@@ -796,12 +801,20 @@ public class SegmentGeneratorConfig implements Serializable {
     _continueOnError = continueOnError;
   }
 
-  public boolean isValidateTimeValue() {
-    return _validateTimeValue;
+  public boolean isRowTimeValueCheck() {
+    return _rowTimeValueCheck;
+  }
+
+  public void setRowTimeValueCheck(boolean rowTimeValueCheck) {
+    _rowTimeValueCheck = rowTimeValueCheck;
+  }
+
+  public boolean isSegmentTimeValueCheck() {
+    return _segmentTimeValueCheck;
   }
 
-  public void setValidateTimeValue(boolean validateTimeValue) {
-    _validateTimeValue = validateTimeValue;
+  public void setSegmentTimeValueCheck(boolean segmentTimeValueCheck) {
+    _segmentTimeValueCheck = segmentTimeValueCheck;
   }
 
   public boolean isOptimizeDictionaryForMetrics() {
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java
index b73649cbeb..0211c4c48f 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java
@@ -53,8 +53,6 @@ public class IndexingConfig extends BaseJsonConfig {
   private SegmentPartitionConfig _segmentPartitionConfig;
   private boolean _aggregateMetrics;
   private boolean _nullHandlingEnabled;
-  private boolean _continueOnError;
-  private boolean _validateTimeValue;
 
   /**
    * If `optimizeDictionaryForMetrics` enabled, dictionary is not created for the metric columns
@@ -285,22 +283,6 @@ public class IndexingConfig extends BaseJsonConfig {
     _nullHandlingEnabled = nullHandlingEnabled;
   }
 
-  public boolean isContinueOnError() {
-    return _continueOnError;
-  }
-
-  public void setContinueOnError(boolean continueOnError) {
-    _continueOnError = continueOnError;
-  }
-
-  public boolean isValidateTimeValue() {
-    return _validateTimeValue;
-  }
-
-  public void setValidateTimeValue(boolean validateTimeValue) {
-    _validateTimeValue = validateTimeValue;
-  }
-
   public boolean isOptimizeDictionaryForMetrics() {
     return _optimizeDictionaryForMetrics;
   }
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/IngestionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/IngestionConfig.java
index ba471d899b..222d85d27a 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/IngestionConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/IngestionConfig.java
@@ -48,6 +48,15 @@ public class IngestionConfig extends BaseJsonConfig {
   @JsonPropertyDescription("Configs related to record aggregation function applied during ingestion")
   private List<AggregationConfig> _aggregationConfigs;
 
+  @JsonPropertyDescription("Configs related to skip any row which has error and continue during ingestion")
+  private boolean _continueOnError;
+
+  @JsonPropertyDescription("Configs related to validate time value for each record during ingestion")
+  private boolean _rowTimeValueCheck = true;
+
+  @JsonPropertyDescription("Configs related to check time value for segment")
+  private boolean _segmentTimeValueCheck = true;
+
   @Deprecated
   public IngestionConfig(@Nullable BatchIngestionConfig batchIngestionConfig,
       @Nullable StreamIngestionConfig streamIngestionConfig, @Nullable FilterConfig filterConfig,
@@ -94,6 +103,18 @@ public class IngestionConfig extends BaseJsonConfig {
     return _aggregationConfigs;
   }
 
+  public boolean isContinueOnError() {
+    return _continueOnError;
+  }
+
+  public boolean isRowTimeValueCheck() {
+    return _rowTimeValueCheck;
+  }
+
+  public boolean isSegmentTimeValueCheck() {
+    return _segmentTimeValueCheck;
+  }
+
   public void setBatchIngestionConfig(BatchIngestionConfig batchIngestionConfig) {
     _batchIngestionConfig = batchIngestionConfig;
   }
@@ -117,4 +138,16 @@ public class IngestionConfig extends BaseJsonConfig {
   public void setAggregationConfigs(List<AggregationConfig> aggregationConfigs) {
     _aggregationConfigs = aggregationConfigs;
   }
+
+  public void setContinueOnError(boolean continueOnError) {
+    _continueOnError = continueOnError;
+  }
+
+  public void setRowTimeValueCheck(boolean rowTimeValueCheck) {
+    _rowTimeValueCheck = rowTimeValueCheck;
+  }
+
+  public void setSegmentTimeValueCheck(boolean segmentTimeValueCheck) {
+    _segmentTimeValueCheck = segmentTimeValueCheck;
+  }
 }
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java
index f2f59a49ef..e884d8155f 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java
@@ -97,8 +97,6 @@ public class TableConfigBuilder {
   private Map<String, String> _streamConfigs;
   private SegmentPartitionConfig _segmentPartitionConfig;
   private boolean _nullHandlingEnabled;
-  private boolean _continueOnError;
-  private boolean _validateTimeValue;
   private List<String> _varLengthDictionaryColumns;
   private List<StarTreeIndexConfig> _starTreeIndexConfigs;
   private List<String> _jsonIndexColumns;
@@ -311,16 +309,6 @@ public class TableConfigBuilder {
     return this;
   }
 
-  public TableConfigBuilder setContinueOnError(boolean continueOnError) {
-    _continueOnError = continueOnError;
-    return this;
-  }
-
-  public TableConfigBuilder setValidateTimeValue(boolean validateTimeValue) {
-    _validateTimeValue = validateTimeValue;
-    return this;
-  }
-
   public TableConfigBuilder setCustomConfig(TableCustomConfig customConfig) {
     _customConfig = customConfig;
     return this;
@@ -432,8 +420,6 @@ public class TableConfigBuilder {
     indexingConfig.setStreamConfigs(_streamConfigs);
     indexingConfig.setSegmentPartitionConfig(_segmentPartitionConfig);
     indexingConfig.setNullHandlingEnabled(_nullHandlingEnabled);
-    indexingConfig.setContinueOnError(_continueOnError);
-    indexingConfig.setValidateTimeValue(_validateTimeValue);
     indexingConfig.setVarLengthDictionaryColumns(_varLengthDictionaryColumns);
     indexingConfig.setStarTreeIndexConfigs(_starTreeIndexConfigs);
     indexingConfig.setJsonIndexColumns(_jsonIndexColumns);


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