You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ja...@apache.org on 2017/12/24 13:26:10 UTC

[14/50] [abbrv] carbondata git commit: [CARBONDATA-1900][Core, processing] Modify loadmetadata to store timestamp long value(in ms) instead of formatted date string for fields "loadStartTime" and "timestamp"

[CARBONDATA-1900][Core,processing] Modify loadmetadata to store timestamp long value(in ms) instead of formatted date string for fields "loadStartTime" and "timestamp"

If the table is moved to environment having different timezone or we change the system current timezone, after IUD operation some of the blocks are not treated as valid blocks.

We should stop writing the loadStartTime and timestamp in dd-MM-yyyy HH:mm:ss:SSS format. We should write the long value of the timestamp

This closes #1666


Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/804ddb76
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/804ddb76
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/804ddb76

Branch: refs/heads/fgdatamap
Commit: 804ddb76fc32eee8a5ab79d8bdd4ede7c64560a1
Parents: 59de7cd
Author: mohammadshahidkhan <mo...@gmail.com>
Authored: Fri Dec 15 14:34:18 2017 +0530
Committer: manishgupta88 <to...@gmail.com>
Committed: Tue Dec 19 15:50:10 2017 +0530

----------------------------------------------------------------------
 .../core/statusmanager/LoadMetadataDetails.java | 90 +++++++++++++-------
 1 file changed, 60 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/804ddb76/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
index bb7fc9d..85602bc 100644
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
@@ -26,6 +26,29 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 
+/*
+  Prior to Carbon 1.3 the the loadMetaData @timestamp and @loadStartTime was stored as
+  as date string format "dd-MM-yyyy HH:mm:ss:SSS". The date string value is specific
+  to the timezone. SO the timestamp in long by the date string will not result into
+  same value if converted to date in different timezone.
+  Json Object of LoadMetaData before CarbonData 1.3
+ |-------------------------------------------------------------------------------------------|
+ | [{"timestamp":"15-12-2017 16:50:31:703","loadStatus":"Success","loadName":"0",            |
+ | "partitionCount":"0","isDeleted":"FALSE","dataSize":"912","indexSize":"700",              |
+ | "updateDeltaEndTimestamp":"","updateDeltaStartTimestamp":"","updateStatusFileName":"",    |
+ | "loadStartTime":"15-12-2017 16:50:27:493","visibility":"true","fileFormat":"COLUMNAR_V3"}]|
+ |-------------------------------------------------------------------------------------------|
+  Fix: As the System.currentTimeMillis() returns the same value irrespective of timezone.
+  So if Carbon stores the long value for @timestamp  and @loadStartTime value then the
+  value will be same irrespective of the timezone.
+  Json Object of LoadMetaData for CarbonData 1.3
+ |-------------------------------------------------------------------------------------------|
+ | [{"timestamp":"1513336827593","loadStatus":"Success","loadName":"0",                      |
+ | "partitionCount":"0","isDeleted":"FALSE","dataSize":"912","indexSize":"700",              |
+ | "updateDeltaEndTimestamp":"","updateDeltaStartTimestamp":"","updateStatusFileName":"",    |
+ | "loadStartTime":"1513336827593","visibility":"true","fileFormat":"COLUMNAR_V3"}]          |
+ |-------------------------------------------------------------------------------------------|
+ */
 public class LoadMetadataDetails implements Serializable {
 
   private static final long serialVersionUID = 1106104914918491724L;
@@ -116,7 +139,7 @@ public class LoadMetadataDetails implements Serializable {
   }
 
   public void setLoadEndTime(long timestamp) {
-    this.timestamp = getTimeStampConvertion(timestamp);;
+    this.timestamp = Long.toString(timestamp);
   }
 
   public SegmentStatus getSegmentStatus() {
@@ -150,7 +173,7 @@ public class LoadMetadataDetails implements Serializable {
    */
   public void setModificationOrdeletionTimesStamp(long modificationOrdeletionTimesStamp) {
     this.modificationOrdeletionTimesStamp =
-        getTimeStampConvertion(modificationOrdeletionTimesStamp);
+        Long.toString(modificationOrdeletionTimesStamp);
   }
 
   /* (non-Javadoc)
@@ -208,28 +231,34 @@ public class LoadMetadataDetails implements Serializable {
    * This method will convert a given timestamp to long value and then to string back
    *
    * @param factTimeStamp
-   * @return
+   * @return Long    TimeStamp value is milliseconds
    */
   private long convertTimeStampToLong(String factTimeStamp) {
-    SimpleDateFormat parser = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP_MILLIS);
-    Date dateToStr = null;
     try {
-      dateToStr = parser.parse(factTimeStamp);
-      return dateToStr.getTime();
-    } catch (ParseException e) {
-      LOGGER.error("Cannot convert" + factTimeStamp + " to Time/Long type value" + e.getMessage());
-      parser = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP);
+      return Long.parseLong(factTimeStamp);
+    } catch (NumberFormatException nf) {
+      SimpleDateFormat parser = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP_MILLIS);
+      // it is the processing for existing table before carbon 1.3
+      Date dateToStr = null;
       try {
-        // if the load is in progress, factTimeStamp will be null, so use current time
-        if (null == factTimeStamp) {
-          return System.currentTimeMillis();
-        }
         dateToStr = parser.parse(factTimeStamp);
         return dateToStr.getTime();
-      } catch (ParseException e1) {
+      } catch (ParseException e) {
         LOGGER
-            .error("Cannot convert" + factTimeStamp + " to Time/Long type value" + e1.getMessage());
-        return 0;
+            .error("Cannot convert" + factTimeStamp + " to Time/Long type value" + e.getMessage());
+        parser = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP);
+        try {
+          // if the load is in progress, factTimeStamp will be null, so use current time
+          if (null == factTimeStamp) {
+            return System.currentTimeMillis();
+          }
+          dateToStr = parser.parse(factTimeStamp);
+          return dateToStr.getTime();
+        } catch (ParseException e1) {
+          LOGGER.error(
+              "Cannot convert" + factTimeStamp + " to Time/Long type value" + e1.getMessage());
+          return 0;
+        }
       }
     }
   }
@@ -238,29 +267,30 @@ public class LoadMetadataDetails implements Serializable {
    * returns load start time as long value
    *
    * @param loadStartTime
-   * @return
+   * @return Long  TimeStamp value is nanoseconds
    */
   public Long getTimeStamp(String loadStartTime) {
-    Date dateToStr = null;
     try {
-      dateToStr = parser.parse(loadStartTime);
-      return dateToStr.getTime() * 1000;
-    } catch (ParseException e) {
-      LOGGER.error("Cannot convert" + loadStartTime + " to Time/Long type value" + e.getMessage());
-      return null;
+      return Long.parseLong(loadStartTime) * 1000L;
+    } catch (NumberFormatException nf) {
+      // it is the processing for existing table before carbon 1.3
+      Date dateToStr = null;
+      try {
+        dateToStr = parser.parse(loadStartTime);
+        return dateToStr.getTime() * 1000;
+      } catch (ParseException e) {
+        LOGGER.error("Cannot convert" + loadStartTime +
+            " to Time/Long type value" + e.getMessage());
+        return null;
+      }
     }
   }
 
-  private String getTimeStampConvertion(long time) {
-    SimpleDateFormat sdf = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP_MILLIS);
-    return sdf.format(time);
-  }
-
   /**
    * @param loadStartTime
    */
   public void setLoadStartTime(long loadStartTime) {
-    this.loadStartTime = getTimeStampConvertion(loadStartTime);
+    this.loadStartTime = Long.toString(loadStartTime);
   }
 
   /**