You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by GitBox <gi...@apache.org> on 2020/08/18 13:10:44 UTC

[GitHub] [carbondata] ShreelekhyaG opened a new pull request #3896: [WIP] Fix load failures due to daylight saving time changes

ShreelekhyaG opened a new pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896


    ### Why is this PR needed?
     1. Fix load failures due to daylight saving time changes.
     2. During load, date/timestamp year values with >4 digit should fail or be null according to bad records action property.
    
    ### What changes were proposed in this PR?
   New property added to setLeniet as true and parse timestampformat.
   Added validation for timestamp range values.
   
       
    ### Does this PR introduce any user interface change?
    - No
    - Yes. (please explain the change and update document)
   
    ### Is any new testcase added?
    - Yes
   
       
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r475346485



##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1592,6 +1592,13 @@ private CarbonCommonConstants() {
 
   public static final String CARBON_LUCENE_INDEX_STOP_WORDS_DEFAULT = "false";
 
+  // Property to enable parsing the timestamp/date data with setLenient = true in load
+  // flow if it fails with parse invalid timestamp data.

Review comment:
       can give the timestamp example and mention DST, so that we will know in future why this property was added.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-677768606


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2076/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [WIP] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r472381642



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -434,20 +434,59 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
   }
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
-    Date dateToStr;
-    DateFormat dateFormatter;
+    Date dateToStr = null;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          LOGGER.info(
+              "Changing setLenient to true for TimeStamp: " + dimensionValue + ". Changing "

Review comment:
       agree. removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-677852439


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3825/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-677856451


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2084/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r476380921



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -444,9 +446,38 @@ private static Object parseTimestamp(String dimensionValue, String dateFormat) {
         dateFormatter = timestampFormatter.get();
       }
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      timeValue = dateToStr.getTime();
+      validateTimeStampRange(timeValue);
+      return timeValue;
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          dateFormatter.setLenient(false);
+          timeValue = dateToStr.getTime();
+          validateTimeStampRange(timeValue);
+          LOGGER.info("Parsed data with lenience as true, setting back to default mode");
+          return timeValue;
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Failed to parse data with lenience as true, setting back to default mode");

Review comment:
       This log is not needed i think, since we are throwing exception




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [WIP] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-675687794


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3777/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-678811108






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-680058834


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2120/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-677798305


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r476183853



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -435,18 +436,48 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
-    DateFormat dateFormatter;
+    DateFormat dateFormatter = null;
+    long timeValue;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);

Review comment:
       I think you dont have to move this line down. as setLenient for timestampFormatter variable is handled during initialisation




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-679920810


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2116/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3896: [WIP] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r472285299



##########
File path: core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
##########
@@ -153,6 +154,12 @@ private boolean validateKeyValue(String key, String value) throws InvalidConfigu
       case ENABLE_UNSAFE_IN_QUERY_EXECUTION:
       case ENABLE_AUTO_LOAD_MERGE:
       case CARBON_PUSH_ROW_FILTERS_FOR_VECTOR:
+      case CARBON_LOAD_SETLENIENT_ENABLE:

Review comment:
       It can be  a fall through case. Can remove line 158-162`

##########
File path: core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
##########
@@ -153,6 +154,12 @@ private boolean validateKeyValue(String key, String value) throws InvalidConfigu
       case ENABLE_UNSAFE_IN_QUERY_EXECUTION:
       case ENABLE_AUTO_LOAD_MERGE:
       case CARBON_PUSH_ROW_FILTERS_FOR_VECTOR:
+      case CARBON_LOAD_SETLENIENT_ENABLE:

Review comment:
       It can be  a fall through case. Can remove line 158-162




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r475377552



##########
File path: integration/spark/src/test/resources/badrecords/invalidTimeStampRange.csv
##########
@@ -0,0 +1,2 @@
+ID,date,starttime,country,name,phonetype,serialname,salary

Review comment:
       ok, made changes.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r473665479



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
##########
@@ -816,10 +816,20 @@ object CarbonDataRDDFactory {
       val partitionByRdd = keyRDD.partitionBy(
         new SegmentPartitioner(segmentIdIndex, segmentUpdateParallelism))
 
+      val conf = SparkSQLUtil.broadCastHadoopConf(sqlContext.sparkSession.sparkContext, hadoopConf)
+      val carbonSessionInfo: CarbonSessionInfo = {

Review comment:
       Instead of this code block from 820-827lines, could have just used 
   `carbonSessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo`
   
   As this object has configuration object  within it, i think, it is better to broadcast and use it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-677783679


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3817/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r476186302



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
##########
@@ -306,7 +315,107 @@ class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterA
     }
   }
 
+  test("test load, update data with setlenient carbon property for daylight " +
+       "saving time from different timezone") {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE, "true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql(
+      """
+           CREATE TABLE IF NOT EXISTS test_time
+           (ID Int, date Date, time Timestamp)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy-MM-dd',
+           'timestampformat'='yyyy-MM-dd HH:mm')
+        """)
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 1"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 11"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 2"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    sql("DROP TABLE test_time")
+    TimeZone.setDefault(defaultTimeZone)

Review comment:
       Can move this to afterAll, as if testcase fails, TimeZone will be left Unset to default




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r476380921



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -444,9 +446,38 @@ private static Object parseTimestamp(String dimensionValue, String dateFormat) {
         dateFormatter = timestampFormatter.get();
       }
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      timeValue = dateToStr.getTime();
+      validateTimeStampRange(timeValue);
+      return timeValue;
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          dateFormatter.setLenient(false);
+          timeValue = dateToStr.getTime();
+          validateTimeStampRange(timeValue);
+          LOGGER.info("Parsed data with lenience as true, setting back to default mode");
+          return timeValue;
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Failed to parse data with lenience as true, setting back to default mode");

Review comment:
       This log is not needed i think, since we are throwing exception




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r477059876



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/util/BadRecordUtil.scala
##########
@@ -68,4 +71,32 @@ object BadRecordUtil {
     badRecordLocation = badRecordLocation + "/" + dbName + "/" + tableName
     FileFactory.deleteAllCarbonFilesOfDir(FileFactory.getCarbonFile(badRecordLocation))
   }
+
+  def createCSV(rows: ListBuffer[Array[String]], csvPath: String): Unit = {
+    val out = new BufferedWriter(new FileWriter(csvPath))

Review comment:
       could not find proper try-with-resource in scala. keeping it the same.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r475346002



##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1592,6 +1592,13 @@ private CarbonCommonConstants() {
 
   public static final String CARBON_LUCENE_INDEX_STOP_WORDS_DEFAULT = "false";
 
+  // Property to enable parsing the data with setLenient = true in load flow if it fails with

Review comment:
       Done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r475494371



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -435,19 +436,48 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
-    DateFormat dateFormatter;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          LOGGER.info("Changing " + dimensionValue + " to " + dateToStr);
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          return validateTimeStampRange(dateToStr.getTime());
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          throw new NumberFormatException(ex.getMessage());
+        }
+      } else {
+        throw new NumberFormatException(e.getMessage());
+      }
+    }
+  }
+
+  private static Long validateTimeStampRange(Long timeValue) {
+    long minValue = DateDirectDictionaryGenerator.MIN_VALUE;
+    long maxValue = DateDirectDictionaryGenerator.MAX_VALUE;
+    if (timeValue < minValue || timeValue > maxValue) {
+      if (LOGGER.isDebugEnabled()) {
+        LOGGER.debug("Value for timestamp type column is not in valid range.");
+      }
+      throw new NumberFormatException("Value for timestamp type column is not in valid range.");

Review comment:
       I think, number format excepiton is ok. Also our exception message shows the correct error string that it is not in the valid range.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r476244994



##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1592,6 +1592,13 @@ private CarbonCommonConstants() {
 
   public static final String CARBON_LUCENE_INDEX_STOP_WORDS_DEFAULT = "false";
 
+  // Property to enable parsing the timestamp/date data with setLenient = true in load
+  // flow if it fails with parse invalid timestamp data.

Review comment:
       ok added example




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r477059942



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -444,9 +446,38 @@ private static Object parseTimestamp(String dimensionValue, String dateFormat) {
         dateFormatter = timestampFormatter.get();
       }
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      timeValue = dateToStr.getTime();
+      validateTimeStampRange(timeValue);
+      return timeValue;
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          dateFormatter.setLenient(false);
+          timeValue = dateToStr.getTime();
+          validateTimeStampRange(timeValue);
+          LOGGER.info("Parsed data with lenience as true, setting back to default mode");
+          return timeValue;
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Failed to parse data with lenience as true, setting back to default mode");
+          throw new NumberFormatException(ex.getMessage());
+        }
+      } else {
+        throw new NumberFormatException(e.getMessage());
+      }
+    }
+  }
+
+  private static void validateTimeStampRange(Long timeValue) {
+    if (timeValue < DateDirectDictionaryGenerator.MIN_VALUE
+        || timeValue > DateDirectDictionaryGenerator.MAX_VALUE) {
+      throw new NumberFormatException(
+          "timestamp column data value: " + timeValue + "is not in valid " + "range of: "

Review comment:
       ok changed

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1592,6 +1592,15 @@ private CarbonCommonConstants() {
 
   public static final String CARBON_LUCENE_INDEX_STOP_WORDS_DEFAULT = "false";
 
+  // Property to enable parsing the timestamp/date data with setLenient = true in load
+  // flow if it fails with parse invalid timestamp data. (example: 1941-03-15 00:00:00
+  // is valid time in Asia/Calcutta zone and is invalid and will fail to parse in Asia/Shanghai
+  // zone as DST is observed and clocks were turned forward 1 hour to 1941-03-15 01:00:00)
+  @CarbonProperty(dynamicConfigurable = true) public static final String

Review comment:
       ok




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r474608789



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -435,19 +436,48 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
-    DateFormat dateFormatter;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          LOGGER.info("Changing " + dimensionValue + " to " + dateToStr);
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          return validateTimeStampRange(dateToStr.getTime());
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          throw new NumberFormatException(ex.getMessage());
+        }
+      } else {
+        throw new NumberFormatException(e.getMessage());
+      }
+    }
+  }
+
+  private static Long validateTimeStampRange(Long timeValue) {
+    long minValue = DateDirectDictionaryGenerator.MIN_VALUE;
+    long maxValue = DateDirectDictionaryGenerator.MAX_VALUE;
+    if (timeValue < minValue || timeValue > maxValue) {
+      if (LOGGER.isDebugEnabled()) {
+        LOGGER.debug("Value for timestamp type column is not in valid range.");
+      }
+      throw new NumberFormatException("Value for timestamp type column is not in valid range.");

Review comment:
       ```suggestion
         throw new NumberFormatException("timestamp column data is not in valid range of: " + DateDirectDictionaryGenerator.MIN_VALUE + " and " + DateDirectDictionaryGenerator.MAX_VALUE );
   ```

##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -435,19 +436,48 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
-    DateFormat dateFormatter;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          LOGGER.info("Changing " + dimensionValue + " to " + dateToStr);
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          return validateTimeStampRange(dateToStr.getTime());
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          throw new NumberFormatException(ex.getMessage());
+        }
+      } else {
+        throw new NumberFormatException(e.getMessage());
+      }
+    }
+  }
+
+  private static Long validateTimeStampRange(Long timeValue) {
+    long minValue = DateDirectDictionaryGenerator.MIN_VALUE;
+    long maxValue = DateDirectDictionaryGenerator.MAX_VALUE;
+    if (timeValue < minValue || timeValue > maxValue) {
+      if (LOGGER.isDebugEnabled()) {

Review comment:
       here debug log is not required, because always the exception is thrown here.

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1592,6 +1592,13 @@ private CarbonCommonConstants() {
 
   public static final String CARBON_LUCENE_INDEX_STOP_WORDS_DEFAULT = "false";
 
+  // Property to enable parsing the data with setLenient = true in load flow if it fails with

Review comment:
       ```suggestion
     // Property to enable parsing the timestamp/date data with setLenient = true in load flow if it fails with
   ```

##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -435,19 +436,48 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
-    DateFormat dateFormatter;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          LOGGER.info("Changing " + dimensionValue + " to " + dateToStr);

Review comment:
       line 456 can be removed, looks this log is not necessary

##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -435,19 +436,48 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
-    DateFormat dateFormatter;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);

Review comment:
       do not print the `dimensionValue ` in log. 
   
   

##########
File path: integration/spark/src/test/resources/badrecords/invalidTimeStampRange.csv
##########
@@ -0,0 +1,2 @@
+ID,date,starttime,country,name,phonetype,serialname,salary

Review comment:
       instead of adding new csv file just for one test case, i suggest delete this file, create this csv file in test case itself and once test is finished, delete the file.

##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -435,19 +436,48 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
-    DateFormat dateFormatter;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          LOGGER.info("Changing " + dimensionValue + " to " + dateToStr);
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          return validateTimeStampRange(dateToStr.getTime());
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          throw new NumberFormatException(ex.getMessage());
+        }
+      } else {
+        throw new NumberFormatException(e.getMessage());
+      }
+    }
+  }
+
+  private static Long validateTimeStampRange(Long timeValue) {
+    long minValue = DateDirectDictionaryGenerator.MIN_VALUE;
+    long maxValue = DateDirectDictionaryGenerator.MAX_VALUE;
+    if (timeValue < minValue || timeValue > maxValue) {
+      if (LOGGER.isDebugEnabled()) {
+        LOGGER.debug("Value for timestamp type column is not in valid range.");
+      }
+      throw new NumberFormatException("Value for timestamp type column is not in valid range.");

Review comment:
       Instead of `NumberFormatException`, can we throw `UnsupportedOperationException`?
   @VenuReddy2103 what you think?

##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -435,19 +436,48 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
-    DateFormat dateFormatter;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          LOGGER.info("Changing " + dimensionValue + " to " + dateToStr);
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          return validateTimeStampRange(dateToStr.getTime());
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          throw new NumberFormatException(ex.getMessage());
+        }
+      } else {
+        throw new NumberFormatException(e.getMessage());
+      }
+    }
+  }
+
+  private static Long validateTimeStampRange(Long timeValue) {

Review comment:
       no need to return Long value here, just make it void and let it just validate the value

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1592,6 +1592,13 @@ private CarbonCommonConstants() {
 
   public static final String CARBON_LUCENE_INDEX_STOP_WORDS_DEFAULT = "false";
 
+  // Property to enable parsing the data with setLenient = true in load flow if it fails with
+  // setLenient = false.

Review comment:
       ```suggestion
     // parse invalid timestamp data.
   ```

##########
File path: integration/spark/src/test/resources/timeStampFormatData3.csv
##########
@@ -0,0 +1,3 @@
+ID,date,starttime,country,name,phonetype,serialname,salary

Review comment:
       same as above

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
##########
@@ -816,10 +816,13 @@ object CarbonDataRDDFactory {
       val partitionByRdd = keyRDD.partitionBy(
         new SegmentPartitioner(segmentIdIndex, segmentUpdateParallelism))
 
+      val carbonSessionInfoBroadcast = sqlContext.sparkSession.sparkContext

Review comment:
       do we really need to broadcast the sessionInfo? How the session parameters are sent to executors in case of normal load flow?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r475378412



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
##########
@@ -816,10 +816,13 @@ object CarbonDataRDDFactory {
       val partitionByRdd = keyRDD.partitionBy(
         new SegmentPartitioner(segmentIdIndex, segmentUpdateParallelism))
 
+      val carbonSessionInfoBroadcast = sqlContext.sparkSession.sparkContext

Review comment:
       In case of normal load flow, NewCarbonDataLoadRDD extends carbonRDD. While initializing, we get carbonSessionInfo from current thread and in compute of carbonRDD we set to the thread by `ThreadLocalSessionInfo.setCarbonSessionInfo(carbonSessionInfo)`. We can either do the same here or broadcast.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r477080281



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -444,9 +446,38 @@ private static Object parseTimestamp(String dimensionValue, String dateFormat) {
         dateFormatter = timestampFormatter.get();
       }
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      timeValue = dateToStr.getTime();
+      validateTimeStampRange(timeValue);
+      return timeValue;
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          dateFormatter.setLenient(true);

Review comment:
       added an example.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3896: [WIP] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r472288683



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
##########
@@ -306,6 +307,39 @@ class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterA
     }
   }
 
+  test("test load, update data with daylight saving time from different timezone") {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE, "true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS t3")
+    sql(
+      """
+           CREATE TABLE IF NOT EXISTS t3
+           (ID Int, date date, starttime Timestamp, country String,
+           name String, phonetype String, serialname String, salary Int)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy/MM/dd',
+           'timestampformat'='yyyy-MM-dd HH:mm')
+        """)
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/timeStampFormatData3.csv' into table t3")
+    sql(s"insert into t3 select 11,'2015-7-23','1941-3-15 00:00:00','china','aaa1','phone197'," +
+        s"'ASD69643',15000")
+    sql("update t3 set (starttime) = ('1941-3-15 00:00:00') where name='aaa2'")
+    checkAnswer(
+      sql("SELECT starttime FROM t3 WHERE ID = 1"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT starttime FROM t3 WHERE ID = 11"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT starttime FROM t3 WHERE ID = 2"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    TimeZone.setDefault(defaultTimeZone)

Review comment:
       Remove `CARBON_LOAD_SETLENIENT_ENABLE` from carbon properies at the end of testcase.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [WIP] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r472380864



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -434,20 +434,59 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
   }
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
-    Date dateToStr;
-    DateFormat dateFormatter;
+    Date dateToStr = null;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          LOGGER.info(
+              "Changing setLenient to true for TimeStamp: " + dimensionValue + ". Changing "
+                  + dimensionValue + " to " + dateToStr);
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          return validateTimeStampRange(dateToStr.getTime());
+        } catch (ParseException ex) {

Review comment:
       ok added

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
##########
@@ -306,6 +307,39 @@ class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterA
     }
   }
 
+  test("test load, update data with daylight saving time from different timezone") {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE, "true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS t3")
+    sql(
+      """
+           CREATE TABLE IF NOT EXISTS t3
+           (ID Int, date date, starttime Timestamp, country String,
+           name String, phonetype String, serialname String, salary Int)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy/MM/dd',
+           'timestampformat'='yyyy-MM-dd HH:mm')
+        """)
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/timeStampFormatData3.csv' into table t3")
+    sql(s"insert into t3 select 11,'2015-7-23','1941-3-15 00:00:00','china','aaa1','phone197'," +
+        s"'ASD69643',15000")
+    sql("update t3 set (starttime) = ('1941-3-15 00:00:00') where name='aaa2'")
+    checkAnswer(
+      sql("SELECT starttime FROM t3 WHERE ID = 1"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT starttime FROM t3 WHERE ID = 11"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT starttime FROM t3 WHERE ID = 2"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    TimeZone.setDefault(defaultTimeZone)

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-681588563


   LGTM


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r477059846



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -444,9 +446,38 @@ private static Object parseTimestamp(String dimensionValue, String dateFormat) {
         dateFormatter = timestampFormatter.get();
       }
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      timeValue = dateToStr.getTime();
+      validateTimeStampRange(timeValue);
+      return timeValue;
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          dateFormatter.setLenient(false);
+          timeValue = dateToStr.getTime();
+          validateTimeStampRange(timeValue);
+          LOGGER.info("Parsed data with lenience as true, setting back to default mode");
+          return timeValue;
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);

Review comment:
       Can move setlenient to false to finally block




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [WIP] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r472381408



##########
File path: core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
##########
@@ -153,6 +154,12 @@ private boolean validateKeyValue(String key, String value) throws InvalidConfigu
       case ENABLE_UNSAFE_IN_QUERY_EXECUTION:
       case ENABLE_AUTO_LOAD_MERGE:
       case CARBON_PUSH_ROW_FILTERS_FOR_VECTOR:
+      case CARBON_LOAD_SETLENIENT_ENABLE:

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [WIP] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-675687520


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2035/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [WIP] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r472294029



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -434,20 +434,59 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
   }
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
-    Date dateToStr;
-    DateFormat dateFormatter;
+    Date dateToStr = null;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          LOGGER.info(
+              "Changing setLenient to true for TimeStamp: " + dimensionValue + ". Changing "
+                  + dimensionValue + " to " + dateToStr);
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          return validateTimeStampRange(dateToStr.getTime());
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          throw new NumberFormatException(ex.getMessage());
+        }
+      } else {
+        throw new NumberFormatException(e.getMessage());
+      }
+    }
+  }
+
+  private static Long validateTimeStampRange(Long timeValue) {
+    SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");

Review comment:
       Here, the `DateDirectDictionaryGenerator.MIN_VALUE`  is ("0001-01-01") which is not equals to timestamp minvalue ("0001-01-01 00:00:00"). As the format is different, will get different long values after parse.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-679916329


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3857/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-680755157


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2132/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r476243946



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
##########
@@ -306,7 +315,107 @@ class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterA
     }
   }
 
+  test("test load, update data with setlenient carbon property for daylight " +
+       "saving time from different timezone") {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE, "true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql(
+      """
+           CREATE TABLE IF NOT EXISTS test_time
+           (ID Int, date Date, time Timestamp)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy-MM-dd',
+           'timestampformat'='yyyy-MM-dd HH:mm')
+        """)
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 1"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 11"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 2"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    sql("DROP TABLE test_time")
+    TimeZone.setDefault(defaultTimeZone)

Review comment:
       ok done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r475716479



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -435,18 +436,48 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
-    DateFormat dateFormatter;
+    DateFormat dateFormatter = null;
+    long timeValue;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      timeValue = dateToStr.getTime();
+      validateTimeStampRange(timeValue);
+      return timeValue;
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true");
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");

Review comment:
       i think these logs not required here, may be you can add only one log say after line 462, saying you had set true as parsing failed for invalid data and parsing finished, something meaningful like this and in line 466, you can say `"failed to parse data with lenience as true, setting back to default"` like this , it will look clean and good

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordActionTest.scala
##########
@@ -270,6 +274,53 @@ class BadRecordActionTest extends QueryTest {
     }
   }
 
+  test("test bad record FAIL with invalid timestamp range") {
+    val csvPath = s"$resourcesPath/badrecords/invalidTimeStampRange.csv"
+    val rows1 = new ListBuffer[Array[String]]
+    rows1 += Array("ID", "date", "time")
+    rows1 += Array("1", "2016-7-24", "342016-7-24 01:02:30")
+    createCSV(rows1, csvPath)
+    sql("DROP TABLE IF EXISTS test_time")
+    sql("""
+           CREATE TABLE IF NOT EXISTS test_time
+           (ID Int, date Date, time Timestamp)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy-MM-dd',
+           'timestampformat'='yyyy-MM-dd HH:mm:ss')
+        """)
+    val exception = intercept[Exception] {
+      sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/badrecords/invalidTimeStampRange.csv' " +
+          s"into table test_time options ('bad_records_action'='fail')")
+    }
+    assert(exception.getMessage
+      .contains(
+        "Data load failed due to bad record: The value with column name time and column data" +
+        " type TIMESTAMP is not a valid TIMESTAMP type.Please enable bad record logger to know" +
+        " the detail reason"))
+    sql("DROP TABLE IF EXISTS test_time")
+    deleteCSVFile(csvPath)
+  }
+
+  def createCSV(rows: ListBuffer[Array[String]], csvPath: String): Unit = {
+    val out = new BufferedWriter(new FileWriter(csvPath))
+    val writer: CSVWriter = new CSVWriter(out)
+
+    for (row <- rows) {
+      writer.writeNext(row)
+    }
+    out.close()
+    writer.close()
+  }
+
+  def deleteCSVFile(csvPath: String): Unit = {

Review comment:
       no need to create new method only for delete, call the file delete API directly in test case as not used in many test case.

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
##########
@@ -306,7 +315,107 @@ class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterA
     }
   }
 
+  test("test load, update data with setlenient carbon property for daylight " +
+       "saving time from different timezone") {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE, "true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql(
+      """
+           CREATE TABLE IF NOT EXISTS test_time
+           (ID Int, date Date, time Timestamp)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy-MM-dd',
+           'timestampformat'='yyyy-MM-dd HH:mm')
+        """)
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(

Review comment:
       put check answers in single line

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
##########
@@ -306,7 +315,107 @@ class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterA
     }
   }
 
+  test("test load, update data with setlenient carbon property for daylight " +
+       "saving time from different timezone") {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE, "true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql(
+      """
+           CREATE TABLE IF NOT EXISTS test_time
+           (ID Int, date Date, time Timestamp)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy-MM-dd',
+           'timestampformat'='yyyy-MM-dd HH:mm')
+        """)
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 1"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 11"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 2"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    sql("DROP TABLE test_time")
+    TimeZone.setDefault(defaultTimeZone)
+    CarbonProperties.getInstance().removeProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE)
+  }
+
+  test("test load, update data with setlenient session level property for daylight " +
+       "saving time from different timezone") {
+    sql("set carbon.load.setlenient.enable = true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql(
+      """
+           CREATE TABLE IF NOT EXISTS test_time
+           (ID Int, date Date, time Timestamp)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy-MM-dd',
+           'timestampformat'='yyyy-MM-dd HH:mm')
+        """)
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 1"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 11"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 2"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    sql("DROP TABLE test_time")
+    TimeZone.setDefault(defaultTimeZone)
+    defaultConfig()
+    sqlContext.sparkSession.conf.unset("carbon.load.setlenient.enable")
+  }
+
+  def generateCSVFile(): Unit = {
+    val rows1 = new ListBuffer[Array[String]]

Review comment:
       ```suggestion
       val rows = new ListBuffer[Array[String]]
   ```

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
##########
@@ -306,7 +315,107 @@ class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterA
     }
   }
 
+  test("test load, update data with setlenient carbon property for daylight " +
+       "saving time from different timezone") {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE, "true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql(
+      """
+           CREATE TABLE IF NOT EXISTS test_time
+           (ID Int, date Date, time Timestamp)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy-MM-dd',
+           'timestampformat'='yyyy-MM-dd HH:mm')
+        """)
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 1"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 11"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 2"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    sql("DROP TABLE test_time")
+    TimeZone.setDefault(defaultTimeZone)
+    CarbonProperties.getInstance().removeProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE)
+  }
+
+  test("test load, update data with setlenient session level property for daylight " +
+       "saving time from different timezone") {
+    sql("set carbon.load.setlenient.enable = true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql(
+      """
+           CREATE TABLE IF NOT EXISTS test_time
+           (ID Int, date Date, time Timestamp)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy-MM-dd',
+           'timestampformat'='yyyy-MM-dd HH:mm')
+        """)
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 1"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 11"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 2"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    sql("DROP TABLE test_time")
+    TimeZone.setDefault(defaultTimeZone)
+    defaultConfig()
+    sqlContext.sparkSession.conf.unset("carbon.load.setlenient.enable")
+  }
+
+  def generateCSVFile(): Unit = {
+    val rows1 = new ListBuffer[Array[String]]
+    rows1 += Array("ID", "date", "time")
+    rows1 += Array("1", "1941-3-15", "1941-3-15 00:00:00")
+    rows1 += Array("2", "2016-7-24", "2016-7-24 01:02:30")
+    createCSV(rows1, csvPath1)
+  }
+
+  def createCSV(rows: ListBuffer[Array[String]], csvPath: String): Unit = {

Review comment:
       instead of duplicate code, copy this method in any test util class, so that later anyone can reuse this.

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordActionTest.scala
##########
@@ -270,6 +274,53 @@ class BadRecordActionTest extends QueryTest {
     }
   }
 
+  test("test bad record FAIL with invalid timestamp range") {
+    val csvPath = s"$resourcesPath/badrecords/invalidTimeStampRange.csv"
+    val rows1 = new ListBuffer[Array[String]]
+    rows1 += Array("ID", "date", "time")
+    rows1 += Array("1", "2016-7-24", "342016-7-24 01:02:30")
+    createCSV(rows1, csvPath)
+    sql("DROP TABLE IF EXISTS test_time")
+    sql("""
+           CREATE TABLE IF NOT EXISTS test_time
+           (ID Int, date Date, time Timestamp)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy-MM-dd',
+           'timestampformat'='yyyy-MM-dd HH:mm:ss')
+        """)
+    val exception = intercept[Exception] {
+      sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/badrecords/invalidTimeStampRange.csv' " +
+          s"into table test_time options ('bad_records_action'='fail')")
+    }
+    assert(exception.getMessage
+      .contains(
+        "Data load failed due to bad record: The value with column name time and column data" +
+        " type TIMESTAMP is not a valid TIMESTAMP type.Please enable bad record logger to know" +
+        " the detail reason"))
+    sql("DROP TABLE IF EXISTS test_time")
+    deleteCSVFile(csvPath)
+  }
+
+  def createCSV(rows: ListBuffer[Array[String]], csvPath: String): Unit = {
+    val out = new BufferedWriter(new FileWriter(csvPath))
+    val writer: CSVWriter = new CSVWriter(out)
+

Review comment:
       remove empty line, and add in try catch for writing and close in finally in a standard way.

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordActionTest.scala
##########
@@ -270,6 +274,53 @@ class BadRecordActionTest extends QueryTest {
     }
   }
 
+  test("test bad record FAIL with invalid timestamp range") {
+    val csvPath = s"$resourcesPath/badrecords/invalidTimeStampRange.csv"
+    val rows1 = new ListBuffer[Array[String]]
+    rows1 += Array("ID", "date", "time")
+    rows1 += Array("1", "2016-7-24", "342016-7-24 01:02:30")
+    createCSV(rows1, csvPath)
+    sql("DROP TABLE IF EXISTS test_time")
+    sql("""

Review comment:
       can format create SQL in just two lines, as we do not worry about scala style in test files. We can avoid duplicate lines or more lines

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
##########
@@ -306,7 +315,107 @@ class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterA
     }
   }
 
+  test("test load, update data with setlenient carbon property for daylight " +
+       "saving time from different timezone") {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE, "true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql(
+      """

Review comment:
       same as above

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
##########
@@ -306,7 +315,107 @@ class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterA
     }
   }
 
+  test("test load, update data with setlenient carbon property for daylight " +
+       "saving time from different timezone") {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE, "true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql(
+      """
+           CREATE TABLE IF NOT EXISTS test_time
+           (ID Int, date Date, time Timestamp)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy-MM-dd',
+           'timestampformat'='yyyy-MM-dd HH:mm')
+        """)
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 1"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 11"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 2"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    sql("DROP TABLE test_time")
+    TimeZone.setDefault(defaultTimeZone)
+    CarbonProperties.getInstance().removeProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE)
+  }
+
+  test("test load, update data with setlenient session level property for daylight " +
+       "saving time from different timezone") {
+    sql("set carbon.load.setlenient.enable = true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql(

Review comment:
       same as above

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
##########
@@ -306,7 +315,107 @@ class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterA
     }
   }
 
+  test("test load, update data with setlenient carbon property for daylight " +
+       "saving time from different timezone") {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE, "true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql(
+      """
+           CREATE TABLE IF NOT EXISTS test_time
+           (ID Int, date Date, time Timestamp)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy-MM-dd',
+           'timestampformat'='yyyy-MM-dd HH:mm')
+        """)
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 1"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 11"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 2"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    sql("DROP TABLE test_time")
+    TimeZone.setDefault(defaultTimeZone)
+    CarbonProperties.getInstance().removeProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE)
+  }
+
+  test("test load, update data with setlenient session level property for daylight " +
+       "saving time from different timezone") {
+    sql("set carbon.load.setlenient.enable = true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql(
+      """
+           CREATE TABLE IF NOT EXISTS test_time
+           (ID Int, date Date, time Timestamp)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy-MM-dd',
+           'timestampformat'='yyyy-MM-dd HH:mm')
+        """)
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(

Review comment:
       same as above

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
##########
@@ -306,7 +315,107 @@ class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterA
     }
   }
 
+  test("test load, update data with setlenient carbon property for daylight " +
+       "saving time from different timezone") {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE, "true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql(
+      """
+           CREATE TABLE IF NOT EXISTS test_time
+           (ID Int, date Date, time Timestamp)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy-MM-dd',
+           'timestampformat'='yyyy-MM-dd HH:mm')
+        """)
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 1"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 11"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 2"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    sql("DROP TABLE test_time")
+    TimeZone.setDefault(defaultTimeZone)
+    CarbonProperties.getInstance().removeProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE)
+  }
+
+  test("test load, update data with setlenient session level property for daylight " +
+       "saving time from different timezone") {
+    sql("set carbon.load.setlenient.enable = true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql(
+      """
+           CREATE TABLE IF NOT EXISTS test_time
+           (ID Int, date Date, time Timestamp)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy-MM-dd',
+           'timestampformat'='yyyy-MM-dd HH:mm')
+        """)
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 1"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 11"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 2"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    sql("DROP TABLE test_time")
+    TimeZone.setDefault(defaultTimeZone)
+    defaultConfig()
+    sqlContext.sparkSession.conf.unset("carbon.load.setlenient.enable")
+  }
+
+  def generateCSVFile(): Unit = {
+    val rows1 = new ListBuffer[Array[String]]
+    rows1 += Array("ID", "date", "time")
+    rows1 += Array("1", "1941-3-15", "1941-3-15 00:00:00")
+    rows1 += Array("2", "2016-7-24", "2016-7-24 01:02:30")
+    createCSV(rows1, csvPath1)
+  }
+
+  def createCSV(rows: ListBuffer[Array[String]], csvPath: String): Unit = {
+    val out = new BufferedWriter(new FileWriter(csvPath))
+    val writer: CSVWriter = new CSVWriter(out)
+

Review comment:
       same comment as other test file, applies to this file. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-680182451


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3865/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [WIP] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-675563624


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3770/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-677642444


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3815/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-677686802


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r475377325



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -435,19 +436,48 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
-    DateFormat dateFormatter;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          LOGGER.info("Changing " + dimensionValue + " to " + dateToStr);
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          return validateTimeStampRange(dateToStr.getTime());
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          throw new NumberFormatException(ex.getMessage());
+        }
+      } else {
+        throw new NumberFormatException(e.getMessage());
+      }
+    }
+  }
+
+  private static Long validateTimeStampRange(Long timeValue) {

Review comment:
       done

##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -435,19 +436,48 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
-    DateFormat dateFormatter;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          LOGGER.info("Changing " + dimensionValue + " to " + dateToStr);
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          return validateTimeStampRange(dateToStr.getTime());
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          throw new NumberFormatException(ex.getMessage());
+        }
+      } else {
+        throw new NumberFormatException(e.getMessage());
+      }
+    }
+  }
+
+  private static Long validateTimeStampRange(Long timeValue) {
+    long minValue = DateDirectDictionaryGenerator.MIN_VALUE;
+    long maxValue = DateDirectDictionaryGenerator.MAX_VALUE;
+    if (timeValue < minValue || timeValue > maxValue) {
+      if (LOGGER.isDebugEnabled()) {
+        LOGGER.debug("Value for timestamp type column is not in valid range.");
+      }
+      throw new NumberFormatException("Value for timestamp type column is not in valid range.");

Review comment:
       ok




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r476183063



##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1592,6 +1592,13 @@ private CarbonCommonConstants() {
 
   public static final String CARBON_LUCENE_INDEX_STOP_WORDS_DEFAULT = "false";
 
+  // Property to enable parsing the timestamp/date data with setLenient = true in load
+  // flow if it fails with parse invalid timestamp data.
+  @CarbonProperty(dynamicConfigurable = true)
+  public static final String CARBON_LOAD_SETLENIENT_ENABLE = "carbon.load.setlenient.enable";

Review comment:
       ```suggestion
     public static final String CARBON_LOAD_DATEFORMAT_SETLENIENT_ENABLE = "carbon.load.setlenient.enable";
   ```
   as this property is specific to date format




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r475377258



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -435,19 +436,48 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
-    DateFormat dateFormatter;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);

Review comment:
       ok




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3896: [WIP] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r472327182



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -434,20 +434,59 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
   }
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
-    Date dateToStr;
-    DateFormat dateFormatter;
+    Date dateToStr = null;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          LOGGER.info(
+              "Changing setLenient to true for TimeStamp: " + dimensionValue + ". Changing "

Review comment:
       `Changing setLenient to true for TimeStamp: " + dimensionValue ` is redundant. we have already logged it in line 452. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-676679976


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3800/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r476386840



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/util/BadRecordUtil.scala
##########
@@ -68,4 +71,34 @@ object BadRecordUtil {
     badRecordLocation = badRecordLocation + "/" + dbName + "/" + tableName
     FileFactory.deleteAllCarbonFilesOfDir(FileFactory.getCarbonFile(badRecordLocation))
   }
+
+  def createCSV(rows: ListBuffer[Array[String]], csvPath: String): Unit = {
+    val out = new BufferedWriter(new FileWriter(csvPath))
+    val writer: CSVWriter = new CSVWriter(out)
+    try {
+      for (row <- rows) {
+        writer.writeNext(row)
+      }
+    }
+    catch {
+      case e: Exception =>
+        e.printStackTrace()

Review comment:
       Remove this line




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r476487539



##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1592,6 +1592,15 @@ private CarbonCommonConstants() {
 
   public static final String CARBON_LUCENE_INDEX_STOP_WORDS_DEFAULT = "false";
 
+  // Property to enable parsing the timestamp/date data with setLenient = true in load
+  // flow if it fails with parse invalid timestamp data. (example: 1941-03-15 00:00:00
+  // is valid time in Asia/Calcutta zone and is invalid and will fail to parse in Asia/Shanghai
+  // zone as DST is observed and clocks were turned forward 1 hour to 1941-03-15 01:00:00)
+  @CarbonProperty(dynamicConfigurable = true) public static final String

Review comment:
       `move public static final String` to next line

##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -444,9 +446,38 @@ private static Object parseTimestamp(String dimensionValue, String dateFormat) {
         dateFormatter = timestampFormatter.get();
       }
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      timeValue = dateToStr.getTime();
+      validateTimeStampRange(timeValue);
+      return timeValue;
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          dateFormatter.setLenient(false);
+          timeValue = dateToStr.getTime();
+          validateTimeStampRange(timeValue);
+          LOGGER.info("Parsed data with lenience as true, setting back to default mode");
+          return timeValue;
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Failed to parse data with lenience as true, setting back to default mode");
+          throw new NumberFormatException(ex.getMessage());
+        }
+      } else {
+        throw new NumberFormatException(e.getMessage());
+      }
+    }
+  }
+
+  private static void validateTimeStampRange(Long timeValue) {
+    if (timeValue < DateDirectDictionaryGenerator.MIN_VALUE
+        || timeValue > DateDirectDictionaryGenerator.MAX_VALUE) {
+      throw new NumberFormatException(
+          "timestamp column data value: " + timeValue + "is not in valid " + "range of: "

Review comment:
       `"is not in valid " + "range of: "` correct the formatting here, there are unnecessary concatenations

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/util/BadRecordUtil.scala
##########
@@ -68,4 +71,32 @@ object BadRecordUtil {
     badRecordLocation = badRecordLocation + "/" + dbName + "/" + tableName
     FileFactory.deleteAllCarbonFilesOfDir(FileFactory.getCarbonFile(badRecordLocation))
   }
+
+  def createCSV(rows: ListBuffer[Array[String]], csvPath: String): Unit = {
+    val out = new BufferedWriter(new FileWriter(csvPath))
+    val writer: CSVWriter = new CSVWriter(out)
+    try {
+      for (row <- rows) {
+        writer.writeNext(row)
+      }
+    }
+    catch {
+      case e: Exception =>
+        Assert.fail(e.getMessage)
+    }
+    finally {
+      out.close()
+      writer.close()
+    }
+  }
+
+  def deleteCSVFile(csvPath: String): Unit = {

Review comment:
       i don't think this method is required, just call `FileUtils.forceDelete(new File(csvPath))` in each test case itself and no need of any Assert here, as its not doing any functional validation

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/util/BadRecordUtil.scala
##########
@@ -68,4 +71,32 @@ object BadRecordUtil {
     badRecordLocation = badRecordLocation + "/" + dbName + "/" + tableName
     FileFactory.deleteAllCarbonFilesOfDir(FileFactory.getCarbonFile(badRecordLocation))
   }
+
+  def createCSV(rows: ListBuffer[Array[String]], csvPath: String): Unit = {
+    val out = new BufferedWriter(new FileWriter(csvPath))

Review comment:
       correct the formatting of this method and may be you can use try-with-resource here, you can check once




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r476244302



##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1592,6 +1592,13 @@ private CarbonCommonConstants() {
 
   public static final String CARBON_LUCENE_INDEX_STOP_WORDS_DEFAULT = "false";
 
+  // Property to enable parsing the timestamp/date data with setLenient = true in load
+  // flow if it fails with parse invalid timestamp data.
+  @CarbonProperty(dynamicConfigurable = true)
+  public static final String CARBON_LOAD_SETLENIENT_ENABLE = "carbon.load.setlenient.enable";

Review comment:
       okay




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3896: [WIP] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r472332314



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -434,20 +434,59 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
   }
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
-    Date dateToStr;
-    DateFormat dateFormatter;
+    Date dateToStr = null;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          LOGGER.info(
+              "Changing setLenient to true for TimeStamp: " + dimensionValue + ". Changing "
+                  + dimensionValue + " to " + dateToStr);
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          return validateTimeStampRange(dateToStr.getTime());
+        } catch (ParseException ex) {

Review comment:
       `validateTimeStampRange()` throws `NumberFormatException`. Your would want to do `dateFormatter.setLenient(false);` in that case too..




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r476244034



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -435,18 +436,48 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
-    DateFormat dateFormatter;
+    DateFormat dateFormatter = null;
+    long timeValue;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      timeValue = dateToStr.getTime();
+      validateTimeStampRange(timeValue);
+      return timeValue;
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true");
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          timeValue = dateToStr.getTime();
+          validateTimeStampRange(timeValue);
+          return timeValue;
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          throw new NumberFormatException(ex.getMessage());
+        }
+      } else {
+        throw new NumberFormatException(e.getMessage());
+      }
+    }
+  }
+
+  private static void validateTimeStampRange(Long timeValue) {
+    if (timeValue < DateDirectDictionaryGenerator.MIN_VALUE
+        || timeValue > DateDirectDictionaryGenerator.MAX_VALUE) {
+      throw new NumberFormatException("timestamp column data is not in valid range of: "

Review comment:
       ok




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r476184514



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -435,18 +436,48 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
-    DateFormat dateFormatter;
+    DateFormat dateFormatter = null;
+    long timeValue;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      timeValue = dateToStr.getTime();
+      validateTimeStampRange(timeValue);
+      return timeValue;
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true");
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          timeValue = dateToStr.getTime();
+          validateTimeStampRange(timeValue);
+          return timeValue;
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          throw new NumberFormatException(ex.getMessage());
+        }
+      } else {
+        throw new NumberFormatException(e.getMessage());
+      }
+    }
+  }
+
+  private static void validateTimeStampRange(Long timeValue) {
+    if (timeValue < DateDirectDictionaryGenerator.MIN_VALUE
+        || timeValue > DateDirectDictionaryGenerator.MAX_VALUE) {
+      throw new NumberFormatException("timestamp column data is not in valid range of: "

Review comment:
       Can print timeValue also in the log




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r477080590



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -444,9 +446,38 @@ private static Object parseTimestamp(String dimensionValue, String dateFormat) {
         dateFormatter = timestampFormatter.get();
       }
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      timeValue = dateToStr.getTime();
+      validateTimeStampRange(timeValue);
+      return timeValue;
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          dateFormatter.setLenient(false);
+          timeValue = dateToStr.getTime();
+          validateTimeStampRange(timeValue);
+          LOGGER.info("Parsed data with lenience as true, setting back to default mode");
+          return timeValue;
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);

Review comment:
       ok




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3896: [WIP] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r472278473



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -434,20 +434,59 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
   }
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
-    Date dateToStr;
-    DateFormat dateFormatter;
+    Date dateToStr = null;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          LOGGER.info(
+              "Changing setLenient to true for TimeStamp: " + dimensionValue + ". Changing "
+                  + dimensionValue + " to " + dateToStr);
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          return validateTimeStampRange(dateToStr.getTime());
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          throw new NumberFormatException(ex.getMessage());
+        }
+      } else {
+        throw new NumberFormatException(e.getMessage());
+      }
+    }
+  }
+
+  private static Long validateTimeStampRange(Long timeValue) {
+    SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");

Review comment:
       Instead of creating instance of simpleDateFormat each time, suggest to use existing `DateDirectDictionaryGenerator.MIN_VALUE` and `DateDirectDictionaryGenerator.MAX_VALUE` to validate




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r478153107



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
##########
@@ -306,7 +315,51 @@ class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterA
     }
   }
 
+  test("test load, update data with setlenient carbon property for daylight " +
+       "saving time from different timezone") {
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_LOAD_DATEFORMAT_SETLENIENT_ENABLE, "true")
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql("CREATE TABLE IF NOT EXISTS test_time (ID Int, date Date, time Timestamp) STORED AS carbondata " +
+        "TBLPROPERTIES('dateformat'='yyyy-MM-dd', 'timestampformat'='yyyy-MM-dd HH:mm:ss') ")
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(sql("SELECT time FROM test_time WHERE ID = 1"), Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
+    checkAnswer(sql("SELECT time FROM test_time WHERE ID = 11"), Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
+    checkAnswer(sql("SELECT time FROM test_time WHERE ID = 2"), Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
+    sql("DROP TABLE test_time")
+    CarbonProperties.getInstance().removeProperty(CarbonCommonConstants.CARBON_LOAD_DATEFORMAT_SETLENIENT_ENABLE)
+  }
+
+  test("test load, update data with setlenient session level property for daylight " +
+       "saving time from different timezone") {
+    sql("set carbon.load.dateformat.setlenient.enable = true")
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql("CREATE TABLE IF NOT EXISTS test_time (ID Int, date Date, time Timestamp) STORED AS carbondata " +
+        "TBLPROPERTIES('dateformat'='yyyy-MM-dd', 'timestampformat'='yyyy-MM-dd HH:mm:ss') ")
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(sql("SELECT time FROM test_time WHERE ID = 1"), Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
+    checkAnswer(sql("SELECT time FROM test_time WHERE ID = 11"), Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
+    checkAnswer(sql("SELECT time FROM test_time WHERE ID = 2"), Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
+    sql("DROP TABLE test_time")
+    defaultConfig()
+  }
+
+  def generateCSVFile(): Unit = {
+    val rows = new ListBuffer[Array[String]]
+    rows += Array("ID", "date", "time")
+    rows += Array("1", "1941-3-15", "1941-3-15 00:00:00")
+    rows += Array("2", "2016-7-24", "2016-7-24 01:02:30")
+    BadRecordUtil.createCSV(rows, csvPath)
+  }
+
   override def afterAll {
     sql("DROP TABLE IF EXISTS t3")
+    FileUtils.forceDelete(new File(csvPath))
+    TimeZone.setDefault(defaultTimeZone)

Review comment:
       `afterAll()` is called only once per testcase file. But, have to set  back`TimeZone.setDefault(defaultTimeZone)` at the end of the testcase  where you changed  zone to China/Shanghai.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r476183063



##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1592,6 +1592,13 @@ private CarbonCommonConstants() {
 
   public static final String CARBON_LUCENE_INDEX_STOP_WORDS_DEFAULT = "false";
 
+  // Property to enable parsing the timestamp/date data with setLenient = true in load
+  // flow if it fails with parse invalid timestamp data.
+  @CarbonProperty(dynamicConfigurable = true)
+  public static final String CARBON_LOAD_SETLENIENT_ENABLE = "carbon.load.setlenient.enable";

Review comment:
       ```suggestion
     public static final String CARBON_LOAD_TIMESTAMP_SETLENIENT_ENABLE = "carbon.load.setlenient.enable";
   ```
   as this property is specific to timestamp




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-677681239


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2074/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-675990742


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2044/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r477061165



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -444,9 +446,38 @@ private static Object parseTimestamp(String dimensionValue, String dateFormat) {
         dateFormatter = timestampFormatter.get();
       }
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      timeValue = dateToStr.getTime();
+      validateTimeStampRange(timeValue);
+      return timeValue;
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          dateFormatter.setLenient(true);

Review comment:
       Please add comments in what scenario it is required




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-680785361


   LGTM


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r476387135



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/util/BadRecordUtil.scala
##########
@@ -68,4 +71,34 @@ object BadRecordUtil {
     badRecordLocation = badRecordLocation + "/" + dbName + "/" + tableName
     FileFactory.deleteAllCarbonFilesOfDir(FileFactory.getCarbonFile(badRecordLocation))
   }
+
+  def createCSV(rows: ListBuffer[Array[String]], csvPath: String): Unit = {
+    val out = new BufferedWriter(new FileWriter(csvPath))
+    val writer: CSVWriter = new CSVWriter(out)
+    try {
+      for (row <- rows) {
+        writer.writeNext(row)
+      }
+    }
+    catch {
+      case e: Exception =>
+        e.printStackTrace()
+        Assert.fail(e.getMessage)
+    }
+    finally {
+      out.close()
+      writer.close()
+    }
+  }
+
+  def deleteCSVFile(csvPath: String): Unit = {
+    try {
+      FileUtils.forceDelete(new File(csvPath))
+    }
+    catch {
+      case e: Exception =>
+        e.printStackTrace()

Review comment:
       Remove this line




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-680055956


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3861/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r478153107



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
##########
@@ -306,7 +315,51 @@ class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterA
     }
   }
 
+  test("test load, update data with setlenient carbon property for daylight " +
+       "saving time from different timezone") {
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_LOAD_DATEFORMAT_SETLENIENT_ENABLE, "true")
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql("CREATE TABLE IF NOT EXISTS test_time (ID Int, date Date, time Timestamp) STORED AS carbondata " +
+        "TBLPROPERTIES('dateformat'='yyyy-MM-dd', 'timestampformat'='yyyy-MM-dd HH:mm:ss') ")
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(sql("SELECT time FROM test_time WHERE ID = 1"), Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
+    checkAnswer(sql("SELECT time FROM test_time WHERE ID = 11"), Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
+    checkAnswer(sql("SELECT time FROM test_time WHERE ID = 2"), Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
+    sql("DROP TABLE test_time")
+    CarbonProperties.getInstance().removeProperty(CarbonCommonConstants.CARBON_LOAD_DATEFORMAT_SETLENIENT_ENABLE)
+  }
+
+  test("test load, update data with setlenient session level property for daylight " +
+       "saving time from different timezone") {
+    sql("set carbon.load.dateformat.setlenient.enable = true")
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql("CREATE TABLE IF NOT EXISTS test_time (ID Int, date Date, time Timestamp) STORED AS carbondata " +
+        "TBLPROPERTIES('dateformat'='yyyy-MM-dd', 'timestampformat'='yyyy-MM-dd HH:mm:ss') ")
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(sql("SELECT time FROM test_time WHERE ID = 1"), Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
+    checkAnswer(sql("SELECT time FROM test_time WHERE ID = 11"), Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
+    checkAnswer(sql("SELECT time FROM test_time WHERE ID = 2"), Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
+    sql("DROP TABLE test_time")
+    defaultConfig()
+  }
+
+  def generateCSVFile(): Unit = {
+    val rows = new ListBuffer[Array[String]]
+    rows += Array("ID", "date", "time")
+    rows += Array("1", "1941-3-15", "1941-3-15 00:00:00")
+    rows += Array("2", "2016-7-24", "2016-7-24 01:02:30")
+    BadRecordUtil.createCSV(rows, csvPath)
+  }
+
   override def afterAll {
     sql("DROP TABLE IF EXISTS t3")
+    FileUtils.forceDelete(new File(csvPath))
+    TimeZone.setDefault(defaultTimeZone)

Review comment:
       `afterAll()` is called only once per testcase file. But, have to set  back`TimeZone.setDefault(defaultTimeZone)` at the end of the testcase  where you changed  zone to China/Shanghai.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [WIP] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-675546141


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2028/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] asfgit closed pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-680183052


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2124/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-680780532


   LGTM


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r476269555



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
##########
@@ -306,7 +315,107 @@ class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterA
     }
   }
 
+  test("test load, update data with setlenient carbon property for daylight " +
+       "saving time from different timezone") {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE, "true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql(
+      """
+           CREATE TABLE IF NOT EXISTS test_time
+           (ID Int, date Date, time Timestamp)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy-MM-dd',
+           'timestampformat'='yyyy-MM-dd HH:mm')
+        """)
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 1"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 11"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 2"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    sql("DROP TABLE test_time")
+    TimeZone.setDefault(defaultTimeZone)
+    CarbonProperties.getInstance().removeProperty(
+      CarbonCommonConstants.CARBON_LOAD_SETLENIENT_ENABLE)
+  }
+
+  test("test load, update data with setlenient session level property for daylight " +
+       "saving time from different timezone") {
+    sql("set carbon.load.setlenient.enable = true")
+    val defaultTimeZone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
+    sql("DROP TABLE IF EXISTS test_time")
+    sql(
+      """
+           CREATE TABLE IF NOT EXISTS test_time
+           (ID Int, date Date, time Timestamp)
+           STORED AS carbondata TBLPROPERTIES('dateformat'='yyyy-MM-dd',
+           'timestampformat'='yyyy-MM-dd HH:mm')
+        """)
+    sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv' into table test_time")
+    sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
+    sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 1"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 11"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    checkAnswer(
+      sql("SELECT time FROM test_time WHERE ID = 2"),
+      Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00")))
+    )
+    sql("DROP TABLE test_time")
+    TimeZone.setDefault(defaultTimeZone)
+    defaultConfig()
+    sqlContext.sparkSession.conf.unset("carbon.load.setlenient.enable")
+  }
+
+  def generateCSVFile(): Unit = {
+    val rows1 = new ListBuffer[Array[String]]
+    rows1 += Array("ID", "date", "time")
+    rows1 += Array("1", "1941-3-15", "1941-3-15 00:00:00")
+    rows1 += Array("2", "2016-7-24", "2016-7-24 01:02:30")
+    createCSV(rows1, csvPath1)
+  }
+
+  def createCSV(rows: ListBuffer[Array[String]], csvPath: String): Unit = {

Review comment:
       moved createCSV and deleteCSV to a test util class




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-676013906


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3786/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r477059456



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/util/BadRecordUtil.scala
##########
@@ -68,4 +71,32 @@ object BadRecordUtil {
     badRecordLocation = badRecordLocation + "/" + dbName + "/" + tableName
     FileFactory.deleteAllCarbonFilesOfDir(FileFactory.getCarbonFile(badRecordLocation))
   }
+
+  def createCSV(rows: ListBuffer[Array[String]], csvPath: String): Unit = {
+    val out = new BufferedWriter(new FileWriter(csvPath))
+    val writer: CSVWriter = new CSVWriter(out)
+    try {
+      for (row <- rows) {
+        writer.writeNext(row)
+      }
+    }
+    catch {
+      case e: Exception =>
+        Assert.fail(e.getMessage)
+    }
+    finally {
+      out.close()
+      writer.close()
+    }
+  }
+
+  def deleteCSVFile(csvPath: String): Unit = {

Review comment:
       ok




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-679040734


   LGTM


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-676683099


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2059/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r475377187



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -435,19 +436,48 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
-    DateFormat dateFormatter;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          LOGGER.info("Changing " + dimensionValue + " to " + dateToStr);
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          return validateTimeStampRange(dateToStr.getTime());
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          throw new NumberFormatException(ex.getMessage());
+        }
+      } else {
+        throw new NumberFormatException(e.getMessage());
+      }
+    }
+  }
+
+  private static Long validateTimeStampRange(Long timeValue) {
+    long minValue = DateDirectDictionaryGenerator.MIN_VALUE;
+    long maxValue = DateDirectDictionaryGenerator.MAX_VALUE;
+    if (timeValue < minValue || timeValue > maxValue) {
+      if (LOGGER.isDebugEnabled()) {

Review comment:
       ok, removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#issuecomment-680755507


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3873/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [CARBONDATA-3955] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r474557660



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
##########
@@ -816,10 +816,20 @@ object CarbonDataRDDFactory {
       val partitionByRdd = keyRDD.partitionBy(
         new SegmentPartitioner(segmentIdIndex, segmentUpdateParallelism))
 
+      val conf = SparkSQLUtil.broadCastHadoopConf(sqlContext.sparkSession.sparkContext, hadoopConf)
+      val carbonSessionInfo: CarbonSessionInfo = {

Review comment:
       Done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3896: [WIP] Fix load failures due to daylight saving time changes

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3896:
URL: https://github.com/apache/carbondata/pull/3896#discussion_r472382210



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -434,20 +434,59 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
   }
 
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
-    Date dateToStr;
-    DateFormat dateFormatter;
+    Date dateToStr = null;
+    DateFormat dateFormatter = null;
     try {
       if (null != dateFormat && !dateFormat.trim().isEmpty()) {
         dateFormatter = new SimpleDateFormat(dateFormat);
-        dateFormatter.setLenient(false);
       } else {
         dateFormatter = timestampFormatter.get();
       }
+      dateFormatter.setLenient(false);
       dateToStr = dateFormatter.parse(dimensionValue);
-      return dateToStr.getTime();
+      return validateTimeStampRange(dateToStr.getTime());
     } catch (ParseException e) {
-      throw new NumberFormatException(e.getMessage());
+      // If the parsing fails, try to parse again with setLenient to true if the property is set
+      if (CarbonProperties.getInstance().isSetLenientEnabled()) {
+        try {
+          LOGGER.info("Changing setLenient to true for TimeStamp: " + dimensionValue);
+          dateFormatter.setLenient(true);
+          dateToStr = dateFormatter.parse(dimensionValue);
+          LOGGER.info(
+              "Changing setLenient to true for TimeStamp: " + dimensionValue + ". Changing "
+                  + dimensionValue + " to " + dateToStr);
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          return validateTimeStampRange(dateToStr.getTime());
+        } catch (ParseException ex) {
+          dateFormatter.setLenient(false);
+          LOGGER.info("Changing setLenient back to false");
+          throw new NumberFormatException(ex.getMessage());
+        }
+      } else {
+        throw new NumberFormatException(e.getMessage());
+      }
+    }
+  }
+
+  private static Long validateTimeStampRange(Long timeValue) {
+    SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");

Review comment:
       rechecked and made use of existing value from `DateDirectDictionaryGenerator.MIN_VALUE` and `DateDirectDictionaryGenerator.MAX_VALUE`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org