You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ja...@apache.org on 2019/07/18 01:50:31 UTC

[flink] branch master updated: [FLINK-13280][table-planner-blink] Revert blink changes in DateTimeUtils, and keep it same as flink version

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 60292c0  [FLINK-13280][table-planner-blink] Revert blink changes in DateTimeUtils, and keep it same as flink version
60292c0 is described below

commit 60292c020c35dd1d35f806c169ea1f2afe14a337
Author: Zhenghua Gao <do...@gmail.com>
AuthorDate: Tue Jul 16 15:19:21 2019 +0800

    [FLINK-13280][table-planner-blink] Revert blink changes in DateTimeUtils, and keep it same as flink version
    
    This closes #9124
---
 .../apache/calcite/avatica/util/DateTimeUtils.java | 197 ++------------------
 .../flink/table/codegen/calls/BuiltInMethods.scala |   5 +
 .../table/codegen/calls/ScalarOperatorGens.scala   |   8 +-
 .../flink/table/codegen/calls/StringCallGen.scala  |   2 +-
 .../flink/table/runtime/utils/TestSinkUtil.scala   |  35 +++-
 .../table/runtime/functions/SqlDateTimeUtils.java  | 203 ++++++++++++++++++++-
 6 files changed, 254 insertions(+), 196 deletions(-)

diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/avatica/util/DateTimeUtils.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/avatica/util/DateTimeUtils.java
index 4256f6b..23cadfb 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/avatica/util/DateTimeUtils.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/avatica/util/DateTimeUtils.java
@@ -16,8 +16,6 @@
  */
 package org.apache.calcite.avatica.util;
 
-import org.apache.flink.table.api.ValidationException;
-
 import java.text.DateFormat;
 import java.text.NumberFormat;
 import java.text.ParsePosition;
@@ -69,9 +67,6 @@ public class DateTimeUtils {
 	/** The Java default time zone. */
 	public static final TimeZone DEFAULT_ZONE = TimeZone.getDefault();
 
-	/** User's configured time zone*/
-	private static volatile TimeZone userZone = UTC_ZONE;
-
 	/**
 	 * The number of milliseconds in a second.
 	 */
@@ -96,19 +91,6 @@ public class DateTimeUtils {
 	public static final long MILLIS_PER_DAY = 86400000; // = 24 * 60 * 60 * 1000
 
 	/**
-	 * The number of microseconds in a day.
-	 */
-	public static final long MICROS_PER_DAY = 86400000000L; // = 24 * 60 * 60 * 1000 * 1000
-
-	/**
-	 * The number of seconds in a day.
-	 *
-	 * <p>This is the modulo 'mask' used when converting
-	 * TIMESTAMP values to DATE and TIME values.
-	 */
-	public static final long SECONDS_PER_DAY = 86400; // = 24 * 60 * 60
-
-	/**
 	 * Calendar set to the epoch (1970-01-01 00:00:00 UTC). Useful for
 	 * initializing other values. Calendars are not immutable, so be careful not
 	 * to screw up this object for everyone else.
@@ -122,13 +104,6 @@ public class DateTimeUtils {
 
 	//~ Methods ----------------------------------------------------------------
 
-	public static TimeZone getUserZone() {
-		return userZone;
-	}
-	public static void setUserZone(TimeZone zone) {
-		userZone = zone;
-	}
-
 	/**
 	 * Parses a string using {@link SimpleDateFormat} and a given pattern. This
 	 * method parses a string at the specified parse position and if successful,
@@ -309,48 +284,15 @@ public class DateTimeUtils {
 		return sdf;
 	}
 
-	/** Helper for CAST({timestamp} Or {date} or {time} AS VARCHAR(n)). */
-	public static String unixDateTimeToString(Object o, TimeZone tz) {
-		int offset = tz.getOffset(Calendar.ZONE_OFFSET);
-		if (tz.useDaylightTime()) {
-			offset = tz.getOffset(((java.util.Date) o).getTime());
-		}
-		if (o instanceof Date) {
-			long time = ((Date) o).getTime();
-			time = time + UTC_ZONE.getOffset(time);
-			if (o instanceof java.sql.Date) {
-				return unixDateToString((int) (time / MILLIS_PER_DAY) + offset);
-			}
-			if (o instanceof java.sql.Time) {
-				return unixTimeToString(((int) (time % MILLIS_PER_DAY) + offset) % (int) MILLIS_PER_DAY);
-			}
-			if (o instanceof java.sql.Timestamp) {
-				return unixTimestampToString(time + offset, 3);
-			}
-		}
-		return o.toString();
-	}
-
 	/** Helper for CAST({timestamp} AS VARCHAR(n)). */
 	public static String unixTimestampToString(long timestamp) {
 		return unixTimestampToString(timestamp, 0);
 	}
 
-	/**
-	 *  Returns the formatted timestamp string
-	 *  now, it is only for Timestamp Literal (TimestampString)
-	 *  don't use it for scalar functions
-	 */
 	public static String unixTimestampToString(long timestamp, int precision) {
 		final StringBuilder buf = new StringBuilder(17);
-
-		// Because unixTimeToString does't take offset into account
-		// so, manually adjust the offset here
-		long tzOffset = userZone.getOffset(timestamp);
-		timestamp += tzOffset;
-
-		int date = (int) ((timestamp) / MILLIS_PER_DAY);
-		int time = (int) ((timestamp) % MILLIS_PER_DAY);
+		int date = (int) (timestamp / MILLIS_PER_DAY);
+		int time = (int) (timestamp % MILLIS_PER_DAY);
 		if (time < 0) {
 			--date;
 			time += MILLIS_PER_DAY;
@@ -485,7 +427,7 @@ public class DateTimeUtils {
 	}
 
 	public static int digitCount(int v) {
-		for (int n = 1;; n++) {
+		for (int n = 1; true; n++) {
 			v /= 10;
 			if (v == 0) {
 				return n;
@@ -674,178 +616,63 @@ public class DateTimeUtils {
 		}
 	}
 
-	private static boolean isInteger(String s) {
-		boolean isInt = s.length() > 0;
-		for(int i = 0; i < s.length(); i++)
-		{
-			if(s.charAt(i) < '0' || s.charAt(i) > '9') {
-				isInt = false;
-				break;
-			}
-		}
-		return isInt;
-	}
-
-	private static boolean isLeapYear(int s) {
-		return s % 400 == 0 || (s % 4 == 0 && s % 100 != 0);
-	}
-
-	private static boolean isIllegalDate(int y, int m, int d) {
-		int[] monthOf31Days = new int[]{1, 3, 5, 7, 8, 10, 12};
-		if(y < 0 || y > 9999 || m < 1 || m > 12 || d < 1 || d > 31) {
-			return false;
-		}
-		if(m == 2 && d > 28) {
-			if(!(isLeapYear(y) && d == 29)) {
-				return false;
-			}
-		}
-		if(d == 31) {
-			for(int i: monthOf31Days) {
-				if(i == m) {
-					return true;
-				}
-			}
-			return false;
-		}
-		return true;
-	}
-
-	public static Integer dateStringToUnixDate(String s) {
-		// allow timestamp str to date, e.g. 2017-12-12 09:30:00.0
-		int ws1 = s.indexOf(" ");
-		if (ws1 > 0) {
-			s = s.substring(0, ws1);
-		}
+	public static int dateStringToUnixDate(String s) {
 		int hyphen1 = s.indexOf('-');
 		int y;
 		int m;
 		int d;
 		if (hyphen1 < 0) {
-			if(!isInteger(s.trim())) {
-				return null;
-			}
 			y = Integer.parseInt(s.trim());
 			m = 1;
 			d = 1;
 		} else {
-			if(!isInteger(s.substring(0, hyphen1).trim())) {
-				return null;
-			}
 			y = Integer.parseInt(s.substring(0, hyphen1).trim());
 			final int hyphen2 = s.indexOf('-', hyphen1 + 1);
 			if (hyphen2 < 0) {
-				if(!isInteger(s.substring(hyphen1 + 1).trim())) {
-					return null;
-				}
 				m = Integer.parseInt(s.substring(hyphen1 + 1).trim());
 				d = 1;
 			} else {
-				if(!isInteger(s.substring(hyphen1 + 1, hyphen2).trim())) {
-					return null;
-				}
 				m = Integer.parseInt(s.substring(hyphen1 + 1, hyphen2).trim());
-				if(!isInteger(s.substring(hyphen2 + 1).trim())) {
-					return null;
-				}
 				d = Integer.parseInt(s.substring(hyphen2 + 1).trim());
 			}
 		}
-		if(!isIllegalDate(y, m, d)) {
-			return null;
-		}
 		return ymdToUnixDate(y, m, d);
 	}
 
-	public static Integer timeStringToUnixDate(String v) {
+	public static int timeStringToUnixDate(String v) {
 		return timeStringToUnixDate(v, 0);
 	}
 
-	public static Integer timeStringToUnixDate(String v, int start) {
+	public static int timeStringToUnixDate(String v, int start) {
 		final int colon1 = v.indexOf(':', start);
-		//timezone hh:mm:ss[.ssssss][[+|-]hh:mm:ss]
-		//refer https://www.w3.org/TR/NOTE-datetime
-		int timezoneHour;
-		int timezoneMinute;
 		int hour;
 		int minute;
 		int second;
 		int milli;
-		int operator = -1;
-		int end = v.length();
-		int timezone = v.indexOf('-', start);
-		if (timezone < 0)
-		{
-			timezone = v.indexOf('+', start);
-			operator = 1;
-		}
-		if (timezone < 0) {
-			timezoneHour = 0;
-			timezoneMinute = 0;
-		} else {
-			end = timezone;
-			final int colon3 = v.indexOf(':', timezone);
-			if (colon3 < 0) {
-				if(!isInteger(v.substring(timezone + 1).trim())) {
-					return null;
-				}
-				timezoneHour = Integer.parseInt(v.substring(timezone + 1).trim());
-				timezoneMinute = 0;
-			} else {
-				if(!isInteger(v.substring(timezone + 1, colon3).trim())) {
-					return null;
-				}
-				timezoneHour = Integer.parseInt(v.substring(timezone + 1, colon3).trim());
-				if(!isInteger(v.substring(colon3 + 1).trim())) {
-					return null;
-				}
-				timezoneMinute = Integer.parseInt(v.substring(colon3 + 1).trim());
-			}
-		}
 		if (colon1 < 0) {
-			if(!isInteger(v.substring(start, end).trim())) {
-				return null;
-			}
-			hour = Integer.parseInt(v.substring(start, end).trim());
+			hour = Integer.parseInt(v.trim());
 			minute = 1;
 			second = 1;
 			milli = 0;
 		} else {
-			if(!isInteger(v.substring(start, colon1).trim())) {
-				return null;
-			}
 			hour = Integer.parseInt(v.substring(start, colon1).trim());
 			final int colon2 = v.indexOf(':', colon1 + 1);
 			if (colon2 < 0) {
-				if(!isInteger(v.substring(colon1 + 1, end).trim())) {
-					return null;
-				}
-				minute = Integer.parseInt(v.substring(colon1 + 1, end).trim());
+				minute = Integer.parseInt(v.substring(colon1 + 1).trim());
 				second = 1;
 				milli = 0;
 			} else {
-				if(!isInteger(v.substring(colon1 + 1, colon2).trim())) {
-					return null;
-				}
 				minute = Integer.parseInt(v.substring(colon1 + 1, colon2).trim());
 				int dot = v.indexOf('.', colon2);
 				if (dot < 0) {
-					if(!isInteger(v.substring(colon2 + 1, end).trim())) {
-						return null;
-					}
-					second = Integer.parseInt(v.substring(colon2 + 1, end).trim());
+					second = Integer.parseInt(v.substring(colon2 + 1).trim());
 					milli = 0;
 				} else {
-					if(!isInteger(v.substring(colon2 + 1, dot).trim())) {
-						return null;
-					}
 					second = Integer.parseInt(v.substring(colon2 + 1, dot).trim());
-					milli = parseFraction(v.substring(dot + 1, end).trim(), 100);
+					milli = parseFraction(v.substring(dot + 1).trim(), 100);
 				}
 			}
 		}
-		hour += operator * timezoneHour;
-		minute += operator * timezoneMinute;
 		return hour * (int) MILLIS_PER_HOUR
 			+ minute * (int) MILLIS_PER_MINUTE
 			+ second * (int) MILLIS_PER_SECOND
@@ -976,7 +803,7 @@ public class DateTimeUtils {
 				final int seconds = time / (int) MILLIS_PER_SECOND;
 				return seconds % 60;
 			default:
-				throw new ValidationException("unit " + range + " can not be applied to time variable");
+				throw new AssertionError(range);
 		}
 	}
 
@@ -1133,7 +960,7 @@ public class DateTimeUtils {
 		// Start with an estimate.
 		// Since no month has more than 31 days, the estimate is <= the true value.
 		int m = (date0 - date1) / 31;
-		for (;;) {
+		while (true) {
 			int date2 = addMonths(date1, m);
 			if (date2 >= date0) {
 				return m;
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/BuiltInMethods.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/BuiltInMethods.scala
index 24a454e..4750252 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/BuiltInMethods.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/BuiltInMethods.scala
@@ -485,4 +485,9 @@ object BuiltInMethods {
     "convertTz",
     classOf[String], classOf[String], classOf[String], classOf[String])
 
+  val STRING_TO_DATE = Types.lookupMethod(
+    classOf[SqlDateTimeUtils], "dateStringToUnixDate", classOf[String])
+
+  val STRING_TO_TIME = Types.lookupMethod(
+    classOf[SqlDateTimeUtils], "timeStringToUnixDate", classOf[String])
 }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperatorGens.scala
index 09f5576..af713b4 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperatorGens.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperatorGens.scala
@@ -892,7 +892,7 @@ object ScalarOperatorGens {
         operand,
         resultNullable = true) {
         operandTerm =>
-          s"${qualifyMethod(BuiltInMethod.STRING_TO_DATE.method)}($operandTerm.toString())"
+          s"${qualifyMethod(BuiltInMethods.STRING_TO_DATE)}($operandTerm.toString())"
       }
 
     // String -> Time
@@ -903,7 +903,7 @@ object ScalarOperatorGens {
         operand, 
         resultNullable = true) {
         operandTerm =>
-          s"${qualifyMethod(BuiltInMethod.STRING_TO_TIME.method)}($operandTerm.toString())"
+          s"${qualifyMethod(BuiltInMethods.STRING_TO_TIME)}($operandTerm.toString())"
       }
 
     // String -> Timestamp
@@ -2146,9 +2146,9 @@ object ScalarOperatorGens {
       operandTerm: String): String =
     targetType.getTypeRoot match {
       case DATE =>
-        s"${qualifyMethod(BuiltInMethod.STRING_TO_DATE.method)}($operandTerm.toString())"
+        s"${qualifyMethod(BuiltInMethods.STRING_TO_DATE)}($operandTerm.toString())"
       case TIME_WITHOUT_TIME_ZONE =>
-        s"${qualifyMethod(BuiltInMethod.STRING_TO_TIME.method)}($operandTerm.toString())"
+        s"${qualifyMethod(BuiltInMethods.STRING_TO_TIME)}($operandTerm.toString())"
       case TIMESTAMP_WITHOUT_TIME_ZONE =>
         s"""
            |${qualifyMethod(BuiltInMethods.STRING_TO_TIMESTAMP)}($operandTerm.toString())
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/StringCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/StringCallGen.scala
index 80e1ffb..aae7e87 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/StringCallGen.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/StringCallGen.scala
@@ -184,7 +184,7 @@ object StringCallGen {
       // Date/Time & BinaryString Converting -- start
 
       case TO_DATE if operands.size == 1 && isCharacterString(operands.head.resultType) =>
-        methodGen(BuiltInMethod.STRING_TO_DATE.method)
+        methodGen(BuiltInMethods.STRING_TO_DATE)
 
       case TO_DATE if operands.size == 2 &&
           isCharacterString(operands.head.resultType) &&
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TestSinkUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TestSinkUtil.scala
index 7027e1d..ed73b52 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TestSinkUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TestSinkUtil.scala
@@ -27,11 +27,9 @@ import org.apache.flink.table.types.TypeInfoLogicalTypeConverter
 import org.apache.flink.table.util.TableTestUtil
 import org.apache.flink.types.Row
 import org.apache.flink.util.StringUtils
-
 import org.apache.calcite.avatica.util.DateTimeUtils
-
 import java.sql.{Date, Time, Timestamp}
-import java.util.TimeZone
+import java.util.{Calendar, TimeZone}
 
 import scala.collection.JavaConverters._
 
@@ -57,7 +55,7 @@ object TestSinkUtil {
   def fieldToString(field: Any, tz: TimeZone): String = {
     field match {
       case _: Date | _: Time | _: Timestamp =>
-        DateTimeUtils.unixDateTimeToString(field, tz)
+        unixDateTimeToString(field, tz)
       case _ => StringUtils.arrayAwareToString(field)
     }
   }
@@ -89,4 +87,33 @@ object TestSinkUtil {
     return "Pojo1{" + "ts=" + fieldToString(pojo.ts, tz) + ", msg='" + pojo.msg + "\'}"
   }
 
+  def unixDateTimeToString(value: Any, tz: TimeZone): String = {
+    val offset =
+      if (tz.useDaylightTime()) {
+        tz.getOffset(value.asInstanceOf[java.util.Date].getTime)
+      } else {
+        tz.getOffset(Calendar.ZONE_OFFSET)
+      }
+    val time = value match {
+      case _: java.util.Date =>
+        val origin = value.asInstanceOf[java.util.Date].getTime
+        origin + DateTimeUtils.UTC_ZONE.getOffset(origin)
+    }
+
+    value match {
+      case _: Date =>
+        DateTimeUtils.unixDateToString(
+          (time / DateTimeUtils.MILLIS_PER_DAY).asInstanceOf[Int] + offset)
+      case _: Time =>
+        DateTimeUtils.unixTimeToString(
+          ((time % DateTimeUtils.MILLIS_PER_DAY).asInstanceOf[Int] + offset)
+            % DateTimeUtils.MILLIS_PER_DAY.asInstanceOf[Int]
+        )
+      case _: Timestamp =>
+        DateTimeUtils.unixTimestampToString(time + offset, 3)
+      case _ =>
+        value.toString
+    }
+
+  }
 }
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java
index 496a99c..a63efde 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java
@@ -177,7 +177,6 @@ public class SqlDateTimeUtils {
 		return time + LOCAL_TZ.getOffset(time);
 	}
 
-
 	// --------------------------------------------------------------------------------------------
 	// int/long/double/Decimal --> Date/Timestamp internal representation
 	// --------------------------------------------------------------------------------------------
@@ -202,7 +201,6 @@ public class SqlDateTimeUtils {
 	// String --> String/timestamp conversion
 	// --------------------------------------------------------------------------------------------
 
-
 	// --------------------------------------------------------------------------------------------
 	// String --> Timestamp conversion
 	// --------------------------------------------------------------------------------------------
@@ -1091,4 +1089,205 @@ public class SqlDateTimeUtils {
 	public static long timeToTimestampWithLocalZone(int time, TimeZone tz) {
 		return unixTimestampToLocalDateTime(time).atZone(tz.toZoneId()).toInstant().toEpochMilli();
 	}
+
+	private static boolean isInteger(String s) {
+		boolean isInt = s.length() > 0;
+		for (int i = 0; i < s.length(); i++) {
+			if (s.charAt(i) < '0' || s.charAt(i) > '9') {
+				isInt = false;
+				break;
+			}
+		}
+		return isInt;
+	}
+
+	private static boolean isLeapYear(int s) {
+		return s % 400 == 0 || (s % 4 == 0 && s % 100 != 0);
+	}
+
+	private static boolean isIllegalDate(int y, int m, int d) {
+		int[] monthOf31Days = new int[]{1, 3, 5, 7, 8, 10, 12};
+		if (y < 0 || y > 9999 || m < 1 || m > 12 || d < 1 || d > 31) {
+			return false;
+		}
+		if (m == 2 && d > 28) {
+			if (!(isLeapYear(y) && d == 29)) {
+				return false;
+			}
+		}
+		if (d == 31) {
+			for (int i: monthOf31Days) {
+				if (i == m) {
+					return true;
+				}
+			}
+			return false;
+		}
+		return true;
+	}
+
+	public static Integer dateStringToUnixDate(String s) {
+		// allow timestamp str to date, e.g. 2017-12-12 09:30:00.0
+		int ws1 = s.indexOf(" ");
+		if (ws1 > 0) {
+			s = s.substring(0, ws1);
+		}
+		int hyphen1 = s.indexOf('-');
+		int y;
+		int m;
+		int d;
+		if (hyphen1 < 0) {
+			if (!isInteger(s.trim())) {
+				return null;
+			}
+			y = Integer.parseInt(s.trim());
+			m = 1;
+			d = 1;
+		} else {
+			if (!isInteger(s.substring(0, hyphen1).trim())) {
+				return null;
+			}
+			y = Integer.parseInt(s.substring(0, hyphen1).trim());
+			final int hyphen2 = s.indexOf('-', hyphen1 + 1);
+			if (hyphen2 < 0) {
+				if (!isInteger(s.substring(hyphen1 + 1).trim())) {
+					return null;
+				}
+				m = Integer.parseInt(s.substring(hyphen1 + 1).trim());
+				d = 1;
+			} else {
+				if (!isInteger(s.substring(hyphen1 + 1, hyphen2).trim())) {
+					return null;
+				}
+				m = Integer.parseInt(s.substring(hyphen1 + 1, hyphen2).trim());
+				if (!isInteger(s.substring(hyphen2 + 1).trim())) {
+					return null;
+				}
+				d = Integer.parseInt(s.substring(hyphen2 + 1).trim());
+			}
+		}
+		if (!isIllegalDate(y, m, d)) {
+			return null;
+		}
+		return DateTimeUtils.ymdToUnixDate(y, m, d);
+	}
+
+	public static Integer timeStringToUnixDate(String v) {
+		return timeStringToUnixDate(v, 0);
+	}
+
+	public static Integer timeStringToUnixDate(String v, int start) {
+		final int colon1 = v.indexOf(':', start);
+		//timezone hh:mm:ss[.ssssss][[+|-]hh:mm:ss]
+		//refer https://www.w3.org/TR/NOTE-datetime
+		int timezoneHour;
+		int timezoneMinute;
+		int hour;
+		int minute;
+		int second;
+		int milli;
+		int operator = -1;
+		int end = v.length();
+		int timezone = v.indexOf('-', start);
+		if (timezone < 0) {
+			timezone = v.indexOf('+', start);
+			operator = 1;
+		}
+		if (timezone < 0) {
+			timezoneHour = 0;
+			timezoneMinute = 0;
+		} else {
+			end = timezone;
+			final int colon3 = v.indexOf(':', timezone);
+			if (colon3 < 0) {
+				if (!isInteger(v.substring(timezone + 1).trim())) {
+					return null;
+				}
+				timezoneHour = Integer.parseInt(v.substring(timezone + 1).trim());
+				timezoneMinute = 0;
+			} else {
+				if (!isInteger(v.substring(timezone + 1, colon3).trim())) {
+					return null;
+				}
+				timezoneHour = Integer.parseInt(v.substring(timezone + 1, colon3).trim());
+				if (!isInteger(v.substring(colon3 + 1).trim())) {
+					return null;
+				}
+				timezoneMinute = Integer.parseInt(v.substring(colon3 + 1).trim());
+			}
+		}
+		if (colon1 < 0) {
+			if (!isInteger(v.substring(start, end).trim())) {
+				return null;
+			}
+			hour = Integer.parseInt(v.substring(start, end).trim());
+			minute = 1;
+			second = 1;
+			milli = 0;
+		} else {
+			if (!isInteger(v.substring(start, colon1).trim())) {
+				return null;
+			}
+			hour = Integer.parseInt(v.substring(start, colon1).trim());
+			final int colon2 = v.indexOf(':', colon1 + 1);
+			if (colon2 < 0) {
+				if (!isInteger(v.substring(colon1 + 1, end).trim())) {
+					return null;
+				}
+				minute = Integer.parseInt(v.substring(colon1 + 1, end).trim());
+				second = 1;
+				milli = 0;
+			} else {
+				if (!isInteger(v.substring(colon1 + 1, colon2).trim())) {
+					return null;
+				}
+				minute = Integer.parseInt(v.substring(colon1 + 1, colon2).trim());
+				int dot = v.indexOf('.', colon2);
+				if (dot < 0) {
+					if (!isInteger(v.substring(colon2 + 1, end).trim())) {
+						return null;
+					}
+					second = Integer.parseInt(v.substring(colon2 + 1, end).trim());
+					milli = 0;
+				} else {
+					if (!isInteger(v.substring(colon2 + 1, dot).trim())) {
+						return null;
+					}
+					second = Integer.parseInt(v.substring(colon2 + 1, dot).trim());
+					milli = parseFraction(v.substring(dot + 1, end).trim(), 100);
+				}
+			}
+		}
+		hour += operator * timezoneHour;
+		minute += operator * timezoneMinute;
+		return hour * (int) MILLIS_PER_HOUR
+			+ minute * (int) MILLIS_PER_MINUTE
+			+ second * (int) MILLIS_PER_SECOND
+			+ milli;
+	}
+
+	/** Parses a fraction, multiplying the first character by {@code multiplier},
+	 * the second character by {@code multiplier / 10},
+	 * the third character by {@code multiplier / 100}, and so forth.
+	 *
+	 * <p>For example, {@code parseFraction("1234", 100)} yields {@code 123}. */
+	private static int parseFraction(String v, int multiplier) {
+		int r = 0;
+		for (int i = 0; i < v.length(); i++) {
+			char c = v.charAt(i);
+			int x = c < '0' || c > '9' ? 0 : (c - '0');
+			r += multiplier * x;
+			if (multiplier < 10) {
+				// We're at the last digit. Check for rounding.
+				if (i + 1 < v.length()
+					&& v.charAt(i + 1) >= '5') {
+					++r;
+				}
+				break;
+			}
+			multiplier /= 10;
+		}
+		return r;
+	}
+
 }