diff --git a/fe/src/main/java/org/apache/doris/analysis/DateLiteral.java b/fe/src/main/java/org/apache/doris/analysis/DateLiteral.java index 0a820b34a5e904..ba27d36d386f11 100644 --- a/fe/src/main/java/org/apache/doris/analysis/DateLiteral.java +++ b/fe/src/main/java/org/apache/doris/analysis/DateLiteral.java @@ -17,9 +17,11 @@ package org.apache.doris.analysis; +import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.thrift.TDateLiteral; import org.apache.doris.thrift.TExprNode; @@ -29,16 +31,58 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.LocalDateTime; +import org.joda.time.format.DateTimeFormatter; +import org.joda.time.format.DateTimeFormatterBuilder; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Date; +import java.util.TimeZone; +import java.util.regex.Pattern; public class DateLiteral extends LiteralExpr { private static final Logger LOG = LogManager.getLogger(DateLiteral.class); - private Date date; + + private static final DateLiteral MIN_DATE = new DateLiteral(1900, 1, 1); + private static final DateLiteral MAX_DATE = new DateLiteral(9999, 12, 31); + private static final DateLiteral MIN_DATETIME = + new DateLiteral(1900, 1, 1, 0, 0, 0); + private static final DateLiteral MAX_DATETIME = + new DateLiteral(9999, 12, 31, 23, 59, 59); + + private static DateTimeFormatter DATE_TIME_FORMATTER = null; + private static DateTimeFormatter DATE_FORMATTER = null; + static { + try { + DATE_TIME_FORMATTER = formatBuilder("%Y-%m-%d %H:%i:%s").toFormatter(); + DATE_FORMATTER = formatBuilder("%Y-%m-%d").toFormatter(); + } catch (AnalysisException e) { + LOG.error("invalid date format", e); + System.exit(-1); + } + } + + //Regex used to determine if the TIME field exists int date_format + private static final Pattern HAS_TIME_PART = Pattern.compile("^.*[HhIiklrSsT]+.*$"); + //Date Literal persist type in meta + private enum DateLiteralType { + DATETIME(0), + DATE(1); + + private final int value; + private DateLiteralType(int value) { + this.value = value; + } + + public int value() { + return value; + } + } private DateLiteral() { super(); @@ -48,9 +92,17 @@ public DateLiteral(Type type, boolean isMax) throws AnalysisException { super(); this.type = type; if (type == Type.DATE) { - date = isMax ? TimeUtils.MAX_DATE : TimeUtils.MIN_DATE; + if (isMax) { + copy(MAX_DATE); + } else { + copy(MIN_DATE); + } } else { - date = isMax ? TimeUtils.MAX_DATETIME : TimeUtils.MIN_DATETIME; + if (isMax) { + copy(MAX_DATETIME); + } else { + copy(MIN_DATETIME); + } } analysisDone(); } @@ -61,52 +113,104 @@ public DateLiteral(String s, Type type) throws AnalysisException { analysisDone(); } - protected DateLiteral(DateLiteral other) { - super(other); - date = other.date; + public DateLiteral(long unixTimestamp, TimeZone timeZone, Type type) { + DateTime dt = new DateTime(unixTimestamp, DateTimeZone.forTimeZone(timeZone)); + year = dt.getYear(); + month = dt.getMonthOfYear(); + day = dt.getDayOfMonth(); + hour = dt.getHourOfDay(); + minute = dt.getMinuteOfHour(); + second = dt.getSecondOfMinute(); + if (type == Type.DATE) { + hour = 0; + minute = 0; + second = 0; + this.type = Type.DATE; + } else { + this.type = Type.DATETIME; + } } - @Override - public Expr clone() { - return new DateLiteral(this); + public DateLiteral(long year, long month, long day) { + this.hour = 0; + this.minute = 0; + this.second = 0; + this.year = year; + this.month = month; + this.day = day; + this.type = Type.DATE; } - public static DateLiteral createMinValue(Type type) { - DateLiteral dateLiteral = new DateLiteral(); - dateLiteral.type = type; - if (type == Type.DATE) { - dateLiteral.date = TimeUtils.MIN_DATE; - } else { - dateLiteral.date = TimeUtils.MIN_DATETIME; - } + public DateLiteral(long year, long month, long day, long hour, long minute, long second) { + this.hour = hour; + this.minute = minute; + this.second = second; + this.year = year; + this.month = month; + this.day = day; + this.type = Type.DATETIME; + } - return dateLiteral; + public DateLiteral(DateLiteral other) { + super(other); + hour = other.hour; + minute = other.minute; + second = other.second; + year = other.year; + month = other.month; + day = other.day; + microsecond = other.microsecond; + type = other.type; + } + + public static DateLiteral createMinValue(Type type) throws AnalysisException { + return new DateLiteral(type, false); } private void init(String s, Type type) throws AnalysisException { - Preconditions.checkArgument(type.isDateType()); - date = TimeUtils.parseDate(s, type); - if (type.isScalarType(PrimitiveType.DATE)) { - if (date.compareTo(TimeUtils.MAX_DATE) > 0 || date.compareTo(TimeUtils.MIN_DATE) < 0) { - throw new AnalysisException("Date type column should range from [" + TimeUtils.MIN_DATE + "] to [" - + TimeUtils.MAX_DATE + "]"); - } - } else { - if (date.compareTo(TimeUtils.MAX_DATETIME) > 0 || date.compareTo(TimeUtils.MIN_DATETIME) < 0) { - throw new AnalysisException("Datetime type column should range from [" + TimeUtils.MIN_DATETIME - + "] to [" + TimeUtils.MAX_DATETIME + "]"); + try { + Preconditions.checkArgument(type.isDateType()); + LocalDateTime dateTime; + if (type == Type.DATE) { + dateTime = DATE_FORMATTER.parseLocalDateTime(s); + } else { + dateTime = DATE_TIME_FORMATTER.parseLocalDateTime(s); } + year = dateTime.getYear(); + month = dateTime.getMonthOfYear(); + day = dateTime.getDayOfMonth(); + hour = dateTime.getHourOfDay(); + minute = dateTime.getMinuteOfHour(); + second = dateTime.getSecondOfMinute(); + this.type = type; + } catch (Exception ex) { + throw new AnalysisException("date literal [" + s + "] is valid"); } - this.type = type; + } + + private void copy(DateLiteral other) { + hour = other.hour; + minute = other.minute; + second = other.second; + year = other.year; + month = other.month; + day = other.day; + microsecond = other.microsecond; + type = other.type; + } + + @Override + public Expr clone() { + return new DateLiteral(this); } @Override public boolean isMinValue() { switch (type.getPrimitiveType()) { case DATE: - return this.date.compareTo(TimeUtils.MIN_DATE) == 0; + return this.getStringValue().compareTo(MIN_DATE.getStringValue()) == 0; case DATETIME: - return this.date.compareTo(TimeUtils.MIN_DATETIME) == 0; + return this.getStringValue().compareTo(MIN_DATETIME.getStringValue()) == 0; default: return false; } @@ -114,13 +218,20 @@ public boolean isMinValue() { @Override public Object getRealValue() { - return TimeUtils.dateTransform(date.getTime(), type); + if (type == Type.DATE) { + return year * 16 * 32L + month * 32 + day; + } else if (type == Type.DATETIME) { + return (year * 10000 + month * 100 + day) * 1000000L + hour * 10000 + minute * 100 + second; + } else { + Preconditions.checkState(false, "invalid date type: " + type); + return -1L; + } } // Date column and Datetime column's hash value is not same. @Override public ByteBuffer getHashValue(PrimitiveType type) { - String value = TimeUtils.format(date, type); + String value = getStringValue(); ByteBuffer buffer; try { buffer = ByteBuffer.wrap(value.getBytes("UTF-8")); @@ -150,17 +261,21 @@ public String toSqlImpl() { @Override public String getStringValue() { - return TimeUtils.format(date, type); + if (type == Type.DATE) { + return String.format("%04d-%02d-%02d", year, month, day); + } else { + return String.format("%04d-%02d-%02d %02d:%02d:%02d", year, month, day, hour, minute, second); + } } @Override public long getLongValue() { - return date.getTime(); + return (year * 10000 + month * 100 + day) * 1000000L + hour * 10000 + minute * 100 + second; } @Override public double getDoubleValue() { - return date.getTime(); + return getLongValue(); } @Override @@ -169,31 +284,86 @@ protected void toThrift(TExprNode msg) { msg.date_literal = new TDateLiteral(getStringValue()); } - public Date getValue() { - return date; - } - @Override protected Expr uncheckedCastTo(Type targetType) throws AnalysisException { if (targetType.isDateType()) { return this; } else if (targetType.isStringType()) { - return new StringLiteral(getStringValue()); + return new StringLiteral(getStringValue()); } Preconditions.checkState(false); return this; } + public void castToDate() { + this.type = Type.DATE; + hour = 0; + minute = 0; + second = 0; + } + + private long makePackedDatetime() { + long ymd = ((year * 13 + month) << 5) | day; + long hms = (hour << 12) | (minute << 6) | second; + long packed_datetime = ((ymd << 17) | hms) << 24 + microsecond; + return packed_datetime; + } + @Override public void write(DataOutput out) throws IOException { super.write(out); - out.writeLong(date.getTime()); + //set flag bit in meta, 0 is DATETIME and 1 is DATE + if (this.type == Type.DATETIME) { + out.writeShort(DateLiteralType.DATETIME.value()); + } else if (this.type == Type.DATE) { + out.writeShort(DateLiteralType.DATE.value()); + } else { + throw new IOException("Error date literal type : " + type); + } + out.writeLong(makePackedDatetime()); + } + + private void fromPackedDatetime(long packed_time) { + microsecond = (packed_time % (1L << 24)); + long ymdhms = (packed_time >> 24); + long ymd = ymdhms >> 17; + long hms = ymdhms % (1 << 17); + + day = ymd % (1 << 5); + long ym = ymd >> 5; + month = ym % 13; + year = ym / 13; + year %= 10000; + second = hms % (1 << 6); + minute = (hms >> 6) % (1 << 6); + hour = (hms >> 12); + // set default date literal type to DATETIME + // date literal read from meta will set type by flag bit; + this.type = Type.DATETIME; } @Override public void readFields(DataInput in) throws IOException { super.readFields(in); - date = new Date(in.readLong()); + if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_60) { + short date_literal_type = in.readShort(); + fromPackedDatetime(in.readLong()); + if (date_literal_type == DateLiteralType.DATETIME.value()) { + this.type = Type.DATETIME; + } else if (date_literal_type == DateLiteralType.DATE.value()) { + this.type = Type.DATE; + } else { + throw new IOException("Error date literal type : " + type); + } + } else { + Date date = new Date(in.readLong()); + String date_str = TimeUtils.format(date, Type.DATETIME); + try { + init(date_str, Type.DATETIME); + } catch (AnalysisException ex) { + throw new IOException(ex.getMessage()); + } + } } public static DateLiteral read(DataInput in) throws IOException { @@ -201,4 +371,196 @@ public static DateLiteral read(DataInput in) throws IOException { literal.readFields(in); return literal; } + + public long unixTimestamp(TimeZone timeZone) { + DateTime dt = new DateTime((int) year, (int) month, (int) day, (int) hour, (int) minute, (int) second, + DateTimeZone.forTimeZone(timeZone)); + return dt.getMillis(); + } + + public static DateLiteral dateParser(String date, String pattern) throws AnalysisException { + LocalDateTime dateTime = formatBuilder(pattern).toFormatter().parseLocalDateTime(date); + DateLiteral dateLiteral = new DateLiteral( + dateTime.getYear(), + dateTime.getMonthOfYear(), + dateTime.getDayOfMonth(), + dateTime.getHourOfDay(), + dateTime.getMinuteOfHour(), + dateTime.getSecondOfMinute()); + if(HAS_TIME_PART.matcher(pattern).matches()) { + dateLiteral.setType(Type.DATETIME); + } else { + dateLiteral.setType(Type.DATE); + } + return dateLiteral; + } + + //Return the date stored in the dateliteral as pattern format. + //eg : "%Y-%m-%d" or "%Y-%m-%d %H:%i:%s" + public String dateFormat(String pattern) throws AnalysisException { + DateTimeFormatterBuilder builder = new DateTimeFormatterBuilder(); + if (type == Type.DATE) { + return DATE_FORMATTER.parseLocalDateTime(getStringValue()) + .toString(formatBuilder(pattern).toFormatter()); + } else { + return DATE_TIME_FORMATTER.parseLocalDateTime(getStringValue()) + .toString(formatBuilder(pattern).toFormatter()); + } + } + + private static DateTimeFormatterBuilder formatBuilder(String pattern) throws AnalysisException { + DateTimeFormatterBuilder builder = new DateTimeFormatterBuilder(); + boolean escaped = false; + for (int i = 0; i < pattern.length(); i++) { + char character = pattern.charAt(i); + if (escaped) { + switch (character) { + case 'a': // %a Abbreviated weekday name (Sun..Sat) + builder.appendDayOfWeekShortText(); + break; + case 'b': // %b Abbreviated month name (Jan..Dec) + builder.appendMonthOfYearShortText(); + break; + case 'c': // %c Month, numeric (0..12) + builder.appendMonthOfYear(1); + break; + case 'd': // %d Day of the month, numeric (00..31) + builder.appendDayOfMonth(2); + break; + case 'e': // %e Day of the month, numeric (0..31) + builder.appendDayOfMonth(1); + break; + case 'H': // %H Hour (00..23) + builder.appendHourOfDay(2); + break; + case 'h': // %h Hour (01..12) + case 'I': // %I Hour (01..12) + builder.appendClockhourOfHalfday(2); + break; + case 'i': // %i Minutes, numeric (00..59) + builder.appendMinuteOfHour(2); + break; + case 'j': // %j Day of year (001..366) + builder.appendDayOfYear(3); + break; + case 'k': // %k Hour (0..23) + builder.appendHourOfDay(1); + break; + case 'l': // %l Hour (1..12) + builder.appendClockhourOfHalfday(1); + break; + case 'M': // %M Month name (January..December) + builder.appendMonthOfYearText(); + break; + case 'm': // %m Month, numeric (00..12) + builder.appendMonthOfYear(2); + break; + case 'p': // %p AM or PM + builder.appendHalfdayOfDayText(); + break; + case 'r': // %r Time, 12-hour (hh:mm:ss followed by AM or PM) + builder.appendClockhourOfHalfday(2) + .appendLiteral(':') + .appendMinuteOfHour(2) + .appendLiteral(':') + .appendSecondOfMinute(2) + .appendLiteral(' ') + .appendHalfdayOfDayText(); + break; + case 'S': // %S Seconds (00..59) + case 's': // %s Seconds (00..59) + builder.appendSecondOfMinute(2); + break; + case 'T': // %T Time, 24-hour (hh:mm:ss) + builder.appendHourOfDay(2) + .appendLiteral(':') + .appendMinuteOfHour(2) + .appendLiteral(':') + .appendSecondOfMinute(2); + break; + case 'v': // %v Week (01..53), where Monday is the first day of the week; used with %x + builder.appendWeekOfWeekyear(2); + break; + case 'x': // %x Year for the week, where Monday is the first day of the week, numeric, four digits; used with %v + builder.appendWeekyear(4, 4); + break; + case 'W': // %W Weekday name (Sunday..Saturday) + builder.appendDayOfWeekText(); + break; + case 'Y': // %Y Year, numeric, four digits + builder.appendYear(4, 4); + break; + case 'y': // %y Year, numeric (two digits) + builder.appendTwoDigitYear(2020); + break; + case 'f': // %f Microseconds (000000..999999) + case 'w': // %w Day of the week (0=Sunday..6=Saturday) + case 'U': // %U Week (00..53), where Sunday is the first day of the week + case 'u': // %u Week (00..53), where Monday is the first day of the week + case 'V': // %V Week (01..53), where Sunday is the first day of the week; used with %X + case 'X': // %X Year for the week where Sunday is the first day of the week, numeric, four digits; used with %V + case 'D': // %D Day of the month with English suffix (0th, 1st, 2nd, 3rd, …) + throw new AnalysisException(String.format("%%%s not supported in date format string", character)); + case '%': // %% A literal "%" character + builder.appendLiteral('%'); + break; + default: // % The literal character represented by + builder.appendLiteral(character); + break; + } + escaped = false; + } else if (character == '%') { + escaped = true; + } else { + builder.appendLiteral(character); + } + } + return builder; + } + + public DateLiteral plusDays(int day) throws AnalysisException { + LocalDateTime dateTime; + DateTimeFormatterBuilder builder = new DateTimeFormatterBuilder(); + if (type == Type.DATE) { + dateTime = DATE_FORMATTER.parseLocalDateTime(getStringValue()).plusDays(day); + } else { + dateTime = DATE_TIME_FORMATTER.parseLocalDateTime(getStringValue()).plusDays(day); + } + DateLiteral dateLiteral = new DateLiteral(dateTime.getYear(), dateTime.getMonthOfYear(), dateTime.getDayOfMonth(), + dateTime.getHourOfDay(), dateTime.getMinuteOfHour(), dateTime.getSecondOfMinute()); + dateLiteral.setType(type); + return dateLiteral; + } + + public long getYear() { + return year; + } + + public long getMonth() { + return month; + } + + public long getDay() { + return day; + } + + public long getHour() { + return hour; + } + + public long getMinute() { + return minute; + } + + public long getSecond() { + return second; + } + + private long year; + private long month; + private long day; + private long hour; + private long minute; + private long second; + private long microsecond; } diff --git a/fe/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java b/fe/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java index 6e3cc297a3cded..3a4d6797653895 100644 --- a/fe/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java +++ b/fe/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java @@ -536,23 +536,7 @@ public void analyzeImpl(Analyzer analyzer) throws AnalysisException { LOG.warn("fn {} not exists", fnName.getFunction()); throw new AnalysisException(getFunctionNotFoundError(collectChildReturnTypes())); } - - if (fnName.getFunction().equals("from_unixtime")) { - // if has only one child, it has default time format: yyyy-MM-dd HH:mm:ss.SSSSSS - if (children.size() > 1) { - final StringLiteral formatExpr = (StringLiteral) children.get(1); - final String dateFormat1 = "yyyy-MM-dd HH:mm:ss"; - final String dateFormat2 = "yyyy-MM-dd"; - if (!formatExpr.getStringValue().equals(dateFormat1) - && !formatExpr.getStringValue().equals(dateFormat2)) { - throw new AnalysisException(new StringBuilder("format does't support, try ") - .append("'").append(dateFormat1).append("'") - .append(" or ") - .append("'").append(dateFormat2).append("'.").toString()); - } - } - } - + if (fn.getFunctionName().getFunction().equals("time_diff")) { fn.getReturnType().getPrimitiveType().setTimeType(); return; diff --git a/fe/src/main/java/org/apache/doris/common/FeConstants.java b/fe/src/main/java/org/apache/doris/common/FeConstants.java index 1c87c9926f49d9..9174ec4c56aa79 100644 --- a/fe/src/main/java/org/apache/doris/common/FeConstants.java +++ b/fe/src/main/java/org/apache/doris/common/FeConstants.java @@ -35,5 +35,5 @@ public class FeConstants { // general model // Current meta data version. Use this version to write journals and image - public static int meta_version = FeMetaVersion.VERSION_59; + public static int meta_version = FeMetaVersion.VERSION_60; } diff --git a/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java b/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java index b997ba259f0853..9e7c42ce2c5357 100644 --- a/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java +++ b/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java @@ -128,4 +128,6 @@ public final class FeMetaVersion { public static final int VERSION_58 = 58; // support strict mode in routine load and stream load public static final int VERSION_59 = 59; + // refactor date literal + public static final int VERSION_60 = 60; } diff --git a/fe/src/main/java/org/apache/doris/common/util/TimeUtils.java b/fe/src/main/java/org/apache/doris/common/util/TimeUtils.java index 7deeaed1e336c7..4e0c41df71d82b 100644 --- a/fe/src/main/java/org/apache/doris/common/util/TimeUtils.java +++ b/fe/src/main/java/org/apache/doris/common/util/TimeUtils.java @@ -27,6 +27,8 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.VariableMgr; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -116,6 +118,16 @@ public static synchronized String getCurrentFormatTime() { return DATETIME_FORMAT.format(new Date()); } + public static TimeZone getTimeZone() { + String timezone; + if (ConnectContext.get() != null) { + timezone = ConnectContext.get().getSessionVariable().getTimeZone(); + } else { + timezone = VariableMgr.getGlobalSessionVariable().getTimeZone(); + } + return TimeZone.getTimeZone(ZoneId.of(timezone, timeZoneAliasMap)); + } + public static String longToTimeString(long timeStamp, SimpleDateFormat dateFormat) { if (timeStamp <= 0L) { return "N/A"; @@ -124,7 +136,10 @@ public static String longToTimeString(long timeStamp, SimpleDateFormat dateForma } public static synchronized String longToTimeString(long timeStamp) { - return longToTimeString(timeStamp, DATETIME_FORMAT); + TimeZone timeZone = getTimeZone(); + SimpleDateFormat dateFormatTimeZone = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + dateFormatTimeZone.setTimeZone(timeZone); + return longToTimeString(timeStamp, dateFormatTimeZone); } public static synchronized Date getTimeAsDate(String timeString) { diff --git a/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java index a2c5bd532d43fd..cd4e6e2eb643ab 100644 --- a/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java +++ b/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java @@ -336,6 +336,9 @@ public TMasterOpResult proxyExecute(TMasterOpRequest request) { if (request.isSetUser_ip()) { ctx.setRemoteIP(request.getUser_ip()); } + if (request.isSetTime_zone()) { + ctx.getSessionVariable().setTimeZone(request.getTime_zone()); + } ctx.setThreadLocalInfo(); diff --git a/fe/src/main/java/org/apache/doris/qe/MasterOpExecutor.java b/fe/src/main/java/org/apache/doris/qe/MasterOpExecutor.java index e2ff93f7319677..e86a8517201c00 100644 --- a/fe/src/main/java/org/apache/doris/qe/MasterOpExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/MasterOpExecutor.java @@ -80,6 +80,7 @@ private void forward() throws Exception { params.setExecMemLimit(ctx.getSessionVariable().getMaxExecMemByte()); params.setQueryTimeout(ctx.getSessionVariable().getQueryTimeoutS()); params.setUser_ip(ctx.getRemoteIP()); + params.setTime_zone(ctx.getSessionVariable().getTimeZone()); LOG.info("Forward statement {} to Master {}", ctx.getStmtId(), thriftAddress); diff --git a/fe/src/main/java/org/apache/doris/rewrite/FEFunctions.java b/fe/src/main/java/org/apache/doris/rewrite/FEFunctions.java index dda78b96cdbefa..2a8705896eb609 100644 --- a/fe/src/main/java/org/apache/doris/rewrite/FEFunctions.java +++ b/fe/src/main/java/org/apache/doris/rewrite/FEFunctions.java @@ -26,23 +26,13 @@ import org.apache.doris.analysis.StringLiteral; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; - import com.google.common.base.Preconditions; - -import org.apache.commons.lang.time.DateFormatUtils; -import org.apache.commons.lang.time.DateUtils; +import org.apache.doris.common.util.TimeUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.joda.time.format.DateTimeFormatter; -import org.joda.time.format.DateTimeFormatterBuilder; import java.math.BigDecimal; import java.math.BigInteger; -import java.text.ParseException; -import java.text.SimpleDateFormat; -import java.util.Calendar; -import java.util.Date; -import java.util.Locale; /** * compute functions in FE. @@ -56,28 +46,26 @@ public class FEFunctions { */ @FEFunction(name = "timediff", argTypes = { "DATETIME", "DATETIME" }, returnType = "TIME") public static FloatLiteral timeDiff(LiteralExpr first, LiteralExpr second) throws AnalysisException { - long timediff = (getTime(first) - getTime(second)) / 1000; - return new FloatLiteral((double)timediff, Type.TIME); + long firstTimestamp = ((DateLiteral) first).unixTimestamp(TimeUtils.getTimeZone()); + long secondTimestamp = ((DateLiteral) second).unixTimestamp(TimeUtils.getTimeZone()); + return new FloatLiteral((double) (firstTimestamp - secondTimestamp) / 1000, Type.TIME); } @FEFunction(name = "datediff", argTypes = { "DATETIME", "DATETIME" }, returnType = "INT") public static IntLiteral dateDiff(LiteralExpr first, LiteralExpr second) throws AnalysisException { - SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd"); - try { - // DATEDIFF function only uses the date part for calculations and ignores the time part - long diff = sdf.parse(first.getStringValue()).getTime() - sdf.parse(second.getStringValue()).getTime(); - long datediff = diff / 1000 / 60 / 60 / 24; - return new IntLiteral(datediff, Type.INT); - } catch (ParseException e) { - throw new AnalysisException(e.getLocalizedMessage()); - } + DateLiteral firstDate = ((DateLiteral) first); + DateLiteral secondDate = ((DateLiteral) second); + // DATEDIFF function only uses the date part for calculations and ignores the time part + firstDate.castToDate(); + secondDate.castToDate(); + long datediff = (firstDate.unixTimestamp(TimeUtils.getTimeZone()) - secondDate.unixTimestamp(TimeUtils.getTimeZone())) / 1000 / 60 / 60 / 24; + return new IntLiteral(datediff, Type.INT); } @FEFunction(name = "date_add", argTypes = { "DATETIME", "INT" }, returnType = "DATETIME") public static DateLiteral dateAdd(LiteralExpr date, LiteralExpr day) throws AnalysisException { - Date d = new Date(getTime(date)); - d = DateUtils.addDays(d, (int) day.getLongValue()); - return new DateLiteral(DateFormatUtils.format(d, "yyyy-MM-dd HH:mm:ss"), Type.DATETIME); + DateLiteral dateLiteral = (DateLiteral) date; + return dateLiteral.plusDays((int) day.getLongValue()); } @FEFunction(name = "adddate", argTypes = { "DATETIME", "INT" }, returnType = "DATETIME") @@ -92,415 +80,57 @@ public static DateLiteral daysAdd(LiteralExpr date, LiteralExpr day) throws Anal @FEFunction(name = "date_format", argTypes = { "DATETIME", "VARCHAR" }, returnType = "VARCHAR") public static StringLiteral dateFormat(LiteralExpr date, StringLiteral fmtLiteral) throws AnalysisException { - String result = dateFormat(new Date(getTime(date)), fmtLiteral.getStringValue()); + String result = ((DateLiteral) date).dateFormat(fmtLiteral.getStringValue()); return new StringLiteral(result); } @FEFunction(name = "str_to_date", argTypes = { "VARCHAR", "VARCHAR" }, returnType = "DATETIME") public static DateLiteral dateParse(StringLiteral date, StringLiteral fmtLiteral) throws AnalysisException { - boolean hasTimePart = false; - DateTimeFormatterBuilder builder = new DateTimeFormatterBuilder(); - - String formatString = fmtLiteral.getStringValue(); - boolean escaped = false; - for (int i = 0; i < formatString.length(); i++) { - char character = formatString.charAt(i); - - if (escaped) { - switch (character) { - case 'a': // %a Abbreviated weekday name (Sun..Sat) - builder.appendDayOfWeekShortText(); - break; - case 'b': // %b Abbreviated month name (Jan..Dec) - builder.appendMonthOfYearShortText(); - break; - case 'c': // %c Month, numeric (0..12) - builder.appendMonthOfYear(1); - break; - case 'd': // %d Day of the month, numeric (00..31) - builder.appendDayOfMonth(2); - break; - case 'e': // %e Day of the month, numeric (0..31) - builder.appendDayOfMonth(1); - break; - case 'H': // %H Hour (00..23) - builder.appendHourOfDay(2); - hasTimePart = true; - break; - case 'h': // %h Hour (01..12) - case 'I': // %I Hour (01..12) - builder.appendClockhourOfHalfday(2); - hasTimePart = true; - break; - case 'i': // %i Minutes, numeric (00..59) - builder.appendMinuteOfHour(2); - hasTimePart = true; - break; - case 'j': // %j Day of year (001..366) - builder.appendDayOfYear(3); - break; - case 'k': // %k Hour (0..23) - builder.appendHourOfDay(1); - hasTimePart = true; - break; - case 'l': // %l Hour (1..12) - builder.appendClockhourOfHalfday(1); - hasTimePart = true; - break; - case 'M': // %M Month name (January..December) - builder.appendMonthOfYearText(); - break; - case 'm': // %m Month, numeric (00..12) - builder.appendMonthOfYear(2); - break; - case 'p': // %p AM or PM - builder.appendHalfdayOfDayText(); - break; - case 'r': // %r Time, 12-hour (hh:mm:ss followed by AM or PM) - builder.appendClockhourOfHalfday(2) - .appendLiteral(':') - .appendMinuteOfHour(2) - .appendLiteral(':') - .appendSecondOfMinute(2) - .appendLiteral(' ') - .appendHalfdayOfDayText(); - hasTimePart = true; - break; - case 'S': // %S Seconds (00..59) - case 's': // %s Seconds (00..59) - builder.appendSecondOfMinute(2); - hasTimePart = true; - break; - case 'T': // %T Time, 24-hour (hh:mm:ss) - builder.appendHourOfDay(2) - .appendLiteral(':') - .appendMinuteOfHour(2) - .appendLiteral(':') - .appendSecondOfMinute(2); - hasTimePart = true; - break; - case 'v': // %v Week (01..53), where Monday is the first day of the week; used with %x - builder.appendWeekOfWeekyear(2); - break; - case 'x': // %x Year for the week, where Monday is the first day of the week, numeric, four digits; used with %v - builder.appendWeekyear(4, 4); - break; - case 'W': // %W Weekday name (Sunday..Saturday) - builder.appendDayOfWeekText(); - break; - case 'Y': // %Y Year, numeric, four digits - builder.appendYear(4, 4); - break; - case 'y': // %y Year, numeric (two digits) - builder.appendTwoDigitYear(2020); - break; - case 'f': // %f Microseconds (000000..999999) - case 'w': // %w Day of the week (0=Sunday..6=Saturday) - case 'U': // %U Week (00..53), where Sunday is the first day of the week - case 'u': // %u Week (00..53), where Monday is the first day of the week - case 'V': // %V Week (01..53), where Sunday is the first day of the week; used with %X - case 'X': // %X Year for the week where Sunday is the first day of the week, numeric, four digits; used with %V - case 'D': // %D Day of the month with English suffix (0th, 1st, 2nd, 3rd, …) - throw new AnalysisException(String.format("%%%s not supported in date format string", character)); - case '%': // %% A literal "%" character - builder.appendLiteral('%'); - break; - default: // % The literal character represented by - builder.appendLiteral(character); - break; - } - escaped = false; - } else if (character == '%') { - escaped = true; - } else { - builder.appendLiteral(character); - } - } - - Date retDate = new Date(builder.toFormatter().withLocale(Locale.ENGLISH).parseMillis(date.getStringValue())); - if (hasTimePart) { - return new DateLiteral(DateFormatUtils.format(retDate, "yyyy-MM-dd HH:mm:ss"), Type.DATETIME); - } else { - return new DateLiteral(DateFormatUtils.format(retDate, "yyyy-MM-dd"), Type.DATE); - } + return DateLiteral.dateParser(date.getStringValue(), fmtLiteral.getStringValue()); } @FEFunction(name = "date_sub", argTypes = { "DATETIME", "INT" }, returnType = "DATETIME") public static DateLiteral dateSub(LiteralExpr date, LiteralExpr day) throws AnalysisException { - Date d = new Date(getTime(date)); - d = DateUtils.addDays(d, -(int) day.getLongValue()); - return new DateLiteral(DateFormatUtils.format(d, "yyyy-MM-dd HH:mm:ss"), Type.DATETIME); + return dateAdd(date, new IntLiteral(-(int) day.getLongValue())); } @FEFunction(name = "year", argTypes = { "DATETIME" }, returnType = "INT") public static IntLiteral year(LiteralExpr arg) throws AnalysisException { - long timestamp = getTime(arg); - Calendar instance = Calendar.getInstance(); - instance.setTimeInMillis(timestamp); - return new IntLiteral(instance.get(Calendar.YEAR), Type.INT); + return new IntLiteral(((DateLiteral) arg).getYear(), Type.INT); } @FEFunction(name = "month", argTypes = { "DATETIME" }, returnType = "INT") public static IntLiteral month(LiteralExpr arg) throws AnalysisException { - long timestamp = getTime(arg); - Calendar instance = Calendar.getInstance(); - instance.setTimeInMillis(timestamp); - return new IntLiteral(instance.get(Calendar.MONTH) + 1, Type.INT); + return new IntLiteral(((DateLiteral) arg).getMonth(), Type.INT); } @FEFunction(name = "day", argTypes = { "DATETIME" }, returnType = "INT") public static IntLiteral day(LiteralExpr arg) throws AnalysisException { - long timestamp = getTime(arg); - Calendar instance = Calendar.getInstance(); - instance.setTimeInMillis(timestamp); - return new IntLiteral(instance.get(Calendar.DAY_OF_MONTH), Type.INT); + return new IntLiteral(((DateLiteral) arg).getDay(), Type.INT); } @FEFunction(name = "unix_timestamp", argTypes = { "DATETIME" }, returnType = "INT") public static IntLiteral unix_timestamp(LiteralExpr arg) throws AnalysisException { - long timestamp = getTime(arg); - return new IntLiteral(timestamp / 1000, Type.INT); + return new IntLiteral(((DateLiteral) arg).unixTimestamp(TimeUtils.getTimeZone()) / 1000, Type.INT); } @FEFunction(name = "from_unixtime", argTypes = { "INT" }, returnType = "VARCHAR") public static StringLiteral fromUnixTime(LiteralExpr unixTime) throws AnalysisException { - long ts = unixTime.getLongValue(); - // if unixTime < 0 or larger than 9999-12-31 23:59:59, we should return null, throw a exception and let BE process - if (ts < 0 || ts > 253402271999L) { + //if unixTime < 0, we should return null, throw a exception and let BE process + if (unixTime.getLongValue() < 0) { throw new AnalysisException("unixtime should larger than zero"); } - Date date = new Date(unixTime.getLongValue() * 1000); - return new StringLiteral(dateFormat(date, "%Y-%m-%d %H:%i:%S")); + DateLiteral dl = new DateLiteral(unixTime.getLongValue() * 1000, TimeUtils.getTimeZone(), Type.DATETIME); + return new StringLiteral(dl.getStringValue()); } - @FEFunction(name = "from_unixtime", argTypes = { "INT", "VARCHAR" }, returnType = "VARCHAR") public static StringLiteral fromUnixTime(LiteralExpr unixTime, StringLiteral fmtLiteral) throws AnalysisException { - long ts = unixTime.getLongValue(); - // if unixTime < 0 or larger than 9999-12-31 23:59:59, we should return null, throw a exception and let BE process - if (ts < 0 || ts > 253402271999L) { + //if unixTime < 0, we should return null, throw a exception and let BE process + if (unixTime.getLongValue() < 0) { throw new AnalysisException("unixtime should larger than zero"); } - Date date = new Date(unixTime.getLongValue() * 1000); - //currently, doris BE only support "yyyy-MM-dd HH:mm:ss" and "yyyy-MM-dd" format - return new StringLiteral(DateFormatUtils.format(date, fmtLiteral.getStringValue())); - } - - private static long getTime(LiteralExpr expr) throws AnalysisException { - try { - String[] parsePatterns = { "yyyyMMdd", "yyyy-MM-dd", "yyyy-MM-dd HH:mm:ss" }; - long time; - if (expr instanceof DateLiteral) { - time = expr.getLongValue(); - } else { - time = DateUtils.parseDate(expr.getStringValue(), parsePatterns).getTime(); - } - return time; - } catch (ParseException e) { - throw new AnalysisException(e.getLocalizedMessage()); - } - } - - private static int calFirstWeekDay(int year, int firstWeekDay) { - Calendar calendar = Calendar.getInstance(); - calendar.set(year, Calendar.JANUARY,1); - int firstDay = 1; - calendar.set(Calendar.DAY_OF_MONTH, firstDay); - while (calendar.get(Calendar.DAY_OF_WEEK) != firstWeekDay) { - calendar.set(Calendar.DAY_OF_MONTH, ++firstDay); - } - return firstDay; - } - - private static String dateFormat(Date date, String pattern) { - DateTimeFormatterBuilder formatterBuilder = new DateTimeFormatterBuilder(); - Calendar calendar = Calendar.getInstance(); - boolean escaped = false; - for (int i = 0; i < pattern.length(); i++) { - char character = pattern.charAt(i); - if (escaped) { - switch (character) { - case 'a': // %a Abbreviated weekday name (Sun..Sat) - formatterBuilder.appendDayOfWeekShortText(); - break; - case 'b': // %b Abbreviated month name (Jan..Dec) - formatterBuilder.appendMonthOfYearShortText(); - break; - case 'c': // %c Month, numeric (0..12) - formatterBuilder.appendMonthOfYear(1); - break; - case 'd': // %d Day of the month, numeric (00..31) - formatterBuilder.appendDayOfMonth(2); - break; - case 'e': // %e Day of the month, numeric (0..31) - formatterBuilder.appendDayOfMonth(1); - break; - case 'f': // %f Microseconds (000000..999999) - formatterBuilder.appendFractionOfSecond(6, 9); - break; - case 'H': // %H Hour (00..23) - formatterBuilder.appendHourOfDay(2); - break; - case 'h': // %h Hour (01..12) - case 'I': // %I Hour (01..12) - formatterBuilder.appendClockhourOfHalfday(2); - break; - case 'i': // %i Minutes, numeric (00..59) - formatterBuilder.appendMinuteOfHour(2); - break; - case 'j': // %j Day of year (001..366) - formatterBuilder.appendDayOfYear(3); - break; - case 'k': // %k Hour (0..23) - formatterBuilder.appendHourOfDay(1); - break; - case 'l': // %l Hour (1..12) - formatterBuilder.appendClockhourOfHalfday(1); - break; - case 'M': // %M Month name (January..December) - formatterBuilder.appendMonthOfYearText(); - break; - case 'm': // %m Month, numeric (00..12) - formatterBuilder.appendMonthOfYear(2); - break; - case 'p': // %p AM or PM - formatterBuilder.appendHalfdayOfDayText(); - break; - case 'r': // %r Time, 12-hour (hh:mm:ss followed by AM or PM) - formatterBuilder.appendClockhourOfHalfday(2) - .appendLiteral(':') - .appendMinuteOfHour(2) - .appendLiteral(':') - .appendSecondOfMinute(2) - .appendLiteral(' ') - .appendHalfdayOfDayText(); - break; - case 'S': // %S Seconds (00..59) - case 's': // %s Seconds (00..59) - formatterBuilder.appendSecondOfMinute(2); - break; - case 'T': // %T Time, 24-hour (hh:mm:ss) - formatterBuilder.appendHourOfDay(2) - .appendLiteral(':') - .appendMinuteOfHour(2) - .appendLiteral(':') - .appendSecondOfMinute(2); - break; - case 'V': // %V Week (01..53), where Sunday is the first day of the week; used with %X - { - int week; - calendar.setTime(date); - int firstSunday = calFirstWeekDay(calendar.get(Calendar.YEAR), Calendar.SUNDAY); - if (calendar.get(Calendar.DATE) <= 7 && calendar.get(Calendar.MONTH) == Calendar.JANUARY - && calendar.get(Calendar.DATE) >= firstSunday) { - week = 1; - } else { - calendar.add(Calendar.DATE, -7); - week = calendar.get(Calendar.WEEK_OF_YEAR) + - (calFirstWeekDay(calendar.get(Calendar.YEAR), Calendar.SUNDAY) == 1 ? 1 : 0); - } - formatterBuilder.appendLiteral(String.format("%02d", week)); - break; - } - case 'v': // %v Week (01..53), where Monday is the first day of the week; used with %x - formatterBuilder.appendWeekOfWeekyear(2); - break; - case 'X': // %X Year for the week where Sunday is the first day of the week, numeric, four digits; used with %V - calendar.setTime(date); - if(calendar.get(Calendar.MONTH) == Calendar.JANUARY && - calendar.get(Calendar.DATE) < calFirstWeekDay(calendar.get(Calendar.YEAR), Calendar.SUNDAY)) { - formatterBuilder.appendLiteral(String.valueOf(calendar.get(Calendar.YEAR) - 1)); - } else { - formatterBuilder.appendLiteral(String.valueOf(calendar.get(Calendar.YEAR))); - } - break; - case 'x': // %x Year for the week, where Monday is the first day of the week, numeric, four digits; used with %v - formatterBuilder.appendWeekyear(4, 4); - break; - case 'W': // %W Weekday name (Sunday..Saturday) - formatterBuilder.appendDayOfWeekText(); - break; - case 'w': // %w Day of the week (0=Sunday..6=Saturday) - calendar.setTime(date); - calendar.setFirstDayOfWeek(Calendar.SUNDAY); - formatterBuilder.appendLiteral(String.valueOf(calendar.get(Calendar.DAY_OF_WEEK) - 1)); - break; - case 'y': // %y Year, numeric (two digits) - int PIVOT_YEAR = 2020; - formatterBuilder.appendTwoDigitYear(PIVOT_YEAR); - break; - case 'Y': // %Y Year, numeric, four digits - formatterBuilder.appendYear(4, 4); - break; - case 'D': // %D Day of the month with English suffix (0th, 1st, 2nd, 3rd, …) - calendar.setTime(date); - int day = calendar.get(Calendar.DAY_OF_MONTH); - if (day >= 10 && day <= 19) { - formatterBuilder.appendLiteral(String.valueOf(day) + "th"); - } else { - switch (day % 10) { - case 1: - formatterBuilder.appendLiteral(String.valueOf(day) + "st"); - break; - case 2: - formatterBuilder.appendLiteral(String.valueOf(day) + "nd"); - break; - case 3: - formatterBuilder.appendLiteral(String.valueOf(day) + "rd"); - break; - default: - formatterBuilder.appendLiteral(String.valueOf(day) + "th"); - break; - } - } - break; - case 'U': // %U Week (00..53), where Sunday is the first day of the week - calendar.setTime(date); - if (calendar.get(Calendar.DATE) <= 7 && calendar.get(Calendar.MONTH) == Calendar.JANUARY) { - int firstSunday = calFirstWeekDay(calendar.get(Calendar.YEAR), Calendar.SUNDAY); - formatterBuilder.appendLiteral(String.format("%02d", - ((calendar.get(Calendar.DATE) < firstSunday && firstSunday != 1) ? 0 : 1))); - } else { - calendar.add(Calendar.DATE, -7); - calendar.setFirstDayOfWeek(Calendar.SUNDAY); - formatterBuilder.appendLiteral(String.format("%02d", - calendar.get(Calendar.WEEK_OF_YEAR) - + (calFirstWeekDay(calendar.get(Calendar.YEAR), Calendar.SUNDAY) == 1 ? 1 : 0))); - } - break; - case 'u': // %u Week (00..53), where Monday is the first day of the week - { - calendar.setTime(date); - int week; - int firstMonday = calFirstWeekDay(calendar.get(Calendar.YEAR), Calendar.MONDAY); - if (calendar.get(Calendar.DATE) <= 7 && calendar.get(Calendar.MONTH) == Calendar.JANUARY) { - week = (calendar.get(Calendar.DATE) >= firstMonday || firstMonday == 1) ? 1 : 0 ; - week += (firstMonday >= 5 ? 1 : 0); - } else { - calendar.add(Calendar.DATE, -7); - calendar.setFirstDayOfWeek(Calendar.MONDAY); - week = calendar.get(Calendar.WEEK_OF_YEAR) + ((firstMonday >= 5 || firstMonday == 1) ? 1 : 0); - } - formatterBuilder.appendLiteral(String.format("%02d", week)); - break; - } - case '%': // %% A literal “%” character - formatterBuilder.appendLiteral('%'); - break; - default: // % The literal character represented by - formatterBuilder.appendLiteral(character); - break; - } - escaped = false; - } - else if (character == '%') { - escaped = true; - } - else { - formatterBuilder.appendLiteral(character); - } - } - DateTimeFormatter formatter = formatterBuilder.toFormatter(); - return formatter.withLocale(Locale.US).print(date.getTime()); + DateLiteral dl = new DateLiteral(unixTime.getLongValue() * 1000, TimeUtils.getTimeZone(), Type.DATETIME); + return new StringLiteral(dl.dateFormat(fmtLiteral.getStringValue())); } /** diff --git a/fe/src/test/java/org/apache/doris/catalog/PartitionKeyTest.java b/fe/src/test/java/org/apache/doris/catalog/PartitionKeyTest.java index ad1702c4b89097..d6f2f2dc912f1a 100644 --- a/fe/src/test/java/org/apache/doris/catalog/PartitionKeyTest.java +++ b/fe/src/test/java/org/apache/doris/catalog/PartitionKeyTest.java @@ -29,10 +29,20 @@ import java.util.List; import java.util.TimeZone; +import org.apache.doris.common.FeConstants; +import org.easymock.EasyMock; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; - +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +@RunWith(PowerMockRunner.class) +@PowerMockIgnore({ "org.apache.log4j.*", "javax.management.*" }) +@PrepareForTest(Catalog.class) public class PartitionKeyTest { private static List allColumns; @@ -43,6 +53,8 @@ public class PartitionKeyTest { private static Column largeInt; private static Column date; private static Column datetime; + + private Catalog catalog; @BeforeClass public static void setUp() { @@ -143,6 +155,12 @@ public void compareTest() throws AnalysisException { @Test public void testSerialization() throws Exception { + catalog = EasyMock.createMock(Catalog.class); + PowerMock.mockStatic(Catalog.class); + EasyMock.expect(Catalog.getInstance()).andReturn(catalog).anyTimes(); + EasyMock.expect(Catalog.getCurrentCatalogJournalVersion()).andReturn(FeConstants.meta_version).anyTimes(); + PowerMock.replay(Catalog.class); + // 1. Write objects to file File file = new File("./keyRangePartition"); file.createNewFile(); diff --git a/fe/src/test/java/org/apache/doris/common/util/TimeUtilsTest.java b/fe/src/test/java/org/apache/doris/common/util/TimeUtilsTest.java index 49e86f36c686bd..1cbabecb8d9b38 100644 --- a/fe/src/test/java/org/apache/doris/common/util/TimeUtilsTest.java +++ b/fe/src/test/java/org/apache/doris/common/util/TimeUtilsTest.java @@ -132,9 +132,6 @@ public void testDateTrans() throws AnalysisException { DateLiteral datetime = new DateLiteral("2015-03-01 12:00:00", ScalarType.DATETIME); Assert.assertEquals(20150301120000L, datetime.getRealValue()); - - Assert.assertEquals("2015-03-01", TimeUtils.format(date.getValue(), date.getType())); - Assert.assertEquals("2015-03-01 12:00:00", TimeUtils.format(datetime.getValue(), datetime.getType())); } } diff --git a/fe/src/test/java/org/apache/doris/rewrite/FEFunctionsTest.java b/fe/src/test/java/org/apache/doris/rewrite/FEFunctionsTest.java index 01041a67e7b92f..944284201d3227 100644 --- a/fe/src/test/java/org/apache/doris/rewrite/FEFunctionsTest.java +++ b/fe/src/test/java/org/apache/doris/rewrite/FEFunctionsTest.java @@ -31,8 +31,6 @@ import org.junit.Test; import org.junit.rules.ExpectedException; -import java.util.TimeZone; - import static org.junit.Assert.fail; /* @@ -65,48 +63,48 @@ public void dateDiffTest() throws AnalysisException { expectedResult = new IntLiteral(-30); Assert.assertEquals(expectedResult, actualResult); } - + @Test public void dateAddTest() throws AnalysisException { - DateLiteral actualResult = FEFunctions.dateAdd(new StringLiteral("2018-08-08"), new IntLiteral(1)); - DateLiteral expectedResult = new DateLiteral("2018-08-09", Type.DATE); + DateLiteral actualResult = FEFunctions.dateAdd(new DateLiteral("2018-08-08", Type.DATE), new IntLiteral(1)); + DateLiteral expectedResult = new DateLiteral("2018-08-09 00:00:00", Type.DATETIME); Assert.assertEquals(expectedResult, actualResult); - actualResult = FEFunctions.dateAdd(new StringLiteral("2018-08-08"), new IntLiteral(-1)); - expectedResult = new DateLiteral("2018-08-07", Type.DATE); + actualResult = FEFunctions.dateAdd(new DateLiteral("2018-08-08", Type.DATE), new IntLiteral(-1)); + expectedResult = new DateLiteral("2018-08-07 00:00:00", Type.DATETIME); Assert.assertEquals(expectedResult, actualResult); } - + @Test public void addDateTest() throws AnalysisException { - DateLiteral actualResult = FEFunctions.addDate(new StringLiteral("2018-08-08"), new IntLiteral(1)); - DateLiteral expectedResult = new DateLiteral("2018-08-09", Type.DATE); + DateLiteral actualResult = FEFunctions.addDate(new DateLiteral("2018-08-08", Type.DATE), new IntLiteral(1)); + DateLiteral expectedResult = new DateLiteral("2018-08-09 00:00:00", Type.DATETIME); Assert.assertEquals(expectedResult, actualResult); - actualResult = FEFunctions.addDate(new StringLiteral("2018-08-08"), new IntLiteral(-1)); - expectedResult = new DateLiteral("2018-08-07", Type.DATE); + actualResult = FEFunctions.addDate(new DateLiteral("2018-08-08", Type.DATE), new IntLiteral(-1)); + expectedResult = new DateLiteral("2018-08-07 00:00:00", Type.DATETIME); Assert.assertEquals(expectedResult, actualResult); } - + @Test public void daysAddTest() throws AnalysisException { - DateLiteral actualResult = FEFunctions.daysAdd(new StringLiteral("2018-08-08"), new IntLiteral(1)); + DateLiteral actualResult = FEFunctions.daysAdd(new DateLiteral("2018-08-08", Type.DATE), new IntLiteral(1)); DateLiteral expectedResult = new DateLiteral("2018-08-09", Type.DATE); Assert.assertEquals(expectedResult, actualResult); - actualResult = FEFunctions.daysAdd(new StringLiteral("2018-08-08"), new IntLiteral(-1)); + actualResult = FEFunctions.daysAdd(new DateLiteral("2018-08-08", Type.DATE), new IntLiteral(-1)); expectedResult = new DateLiteral("2018-08-07", Type.DATE); Assert.assertEquals(expectedResult, actualResult); } - + @Test public void fromUnixTimeTest() throws AnalysisException { StringLiteral actualResult = FEFunctions.fromUnixTime(new IntLiteral(100000)); StringLiteral expectedResult = new StringLiteral("1970-01-02 11:46:40"); Assert.assertEquals(expectedResult, actualResult); - actualResult = FEFunctions.fromUnixTime(new IntLiteral(100000), new StringLiteral("yyyy-MM-dd")); + actualResult = FEFunctions.fromUnixTime(new IntLiteral(100000), new StringLiteral("%Y-%m-%d")); expectedResult = new StringLiteral("1970-01-02"); Assert.assertEquals(expectedResult, actualResult); @@ -125,11 +123,6 @@ public void fromUnixTimeTestException() throws AnalysisException { @Test public void dateFormatUtilTest() { try { - Assert.assertEquals("19670102,196701,196701,0101", FEFunctions.dateFormat(new DateLiteral("1967-01-02 13:04:05", Type.DATETIME), new StringLiteral("%Y%m%d,%X%V,%x%v,%U%u")).getStringValue()); - Assert.assertEquals("19960105,199553,199601,0001", FEFunctions.dateFormat(new DateLiteral("1996-01-05 13:04:05", Type.DATETIME), new StringLiteral("%Y%m%d,%X%V,%x%v,%U%u")).getStringValue()); - - Assert.assertEquals("2017-01-01,01,00", FEFunctions.dateFormat(new DateLiteral("2017-01-01 13:04:05", Type.DATETIME), new StringLiteral("%Y-%m-%d,%U,%u")).getStringValue()); - Assert.assertEquals("201753,201752,5352", FEFunctions.dateFormat(new DateLiteral("2017-12-31 13:04:05", Type.DATETIME),new StringLiteral("%X%V,%x%v,%U%u")).getStringValue()); DateLiteral testDate = new DateLiteral("2001-01-09 13:04:05", Type.DATETIME); Assert.assertEquals("Tue", FEFunctions.dateFormat(testDate, new StringLiteral("%a")).getStringValue()); @@ -153,7 +146,6 @@ public void dateFormatUtilTest() { Assert.assertEquals("13:04:05", FEFunctions.dateFormat(testDate, new StringLiteral("%T")).getStringValue()); Assert.assertEquals("02", FEFunctions.dateFormat(testDate, new StringLiteral("%v")).getStringValue()); Assert.assertEquals("Tuesday", FEFunctions.dateFormat(testDate, new StringLiteral("%W")).getStringValue()); - Assert.assertEquals("2", FEFunctions.dateFormat(testDate, new StringLiteral("%w")).getStringValue()); Assert.assertEquals("2001", FEFunctions.dateFormat(testDate, new StringLiteral("%Y")).getStringValue()); Assert.assertEquals("01", FEFunctions.dateFormat(testDate, new StringLiteral("%y")).getStringValue()); Assert.assertEquals("%", FEFunctions.dateFormat(testDate, new StringLiteral("%%")).getStringValue()); @@ -161,7 +153,6 @@ public void dateFormatUtilTest() { Assert.assertEquals("g", FEFunctions.dateFormat(testDate, new StringLiteral("%g")).getStringValue()); Assert.assertEquals("4", FEFunctions.dateFormat(testDate, new StringLiteral("%4")).getStringValue()); Assert.assertEquals("2001 02" ,FEFunctions.dateFormat(testDate, new StringLiteral("%x %v")).getStringValue()); - Assert.assertEquals("9th" ,FEFunctions.dateFormat(testDate, new StringLiteral("%D")).getStringValue()); } catch (AnalysisException e) { e.printStackTrace(); } @@ -169,8 +160,6 @@ public void dateFormatUtilTest() { @Test public void dateParseTest() { - TimeZone tz = TimeZone.getTimeZone("Asia/Shanghai"); - TimeZone.setDefault(tz); try { Assert.assertEquals("2013-05-10", FEFunctions.dateParse(new StringLiteral("2013,05,10"), new StringLiteral("%Y,%m,%d")).getStringValue()); Assert.assertEquals("2013-05-17 00:35:10", FEFunctions.dateParse(new StringLiteral("2013-05-17 12:35:10"), new StringLiteral("%Y-%m-%d %h:%i:%s")).getStringValue()); @@ -229,44 +218,44 @@ public void dateParseTest() { @Test public void dateSubTest() throws AnalysisException { - DateLiteral actualResult = FEFunctions.dateSub(new StringLiteral("2018-08-08"), new IntLiteral(1)); + DateLiteral actualResult = FEFunctions.dateSub(new DateLiteral("2018-08-08", Type.DATE), new IntLiteral(1)); DateLiteral expectedResult = new DateLiteral("2018-08-07", Type.DATE); Assert.assertEquals(expectedResult, actualResult); - actualResult = FEFunctions.dateSub(new StringLiteral("2018-08-08"), new IntLiteral(-1)); + actualResult = FEFunctions.dateSub(new DateLiteral("2018-08-08", Type.DATE), new IntLiteral(-1)); expectedResult = new DateLiteral("2018-08-09", Type.DATE); Assert.assertEquals(expectedResult, actualResult); } @Test public void yearTest() throws AnalysisException { - IntLiteral actualResult = FEFunctions.year(new StringLiteral("2018-08-08")); - IntLiteral expectedResult = new IntLiteral(2018); + IntLiteral actualResult = FEFunctions.year(new DateLiteral("2018-08-08", Type.DATE)); + IntLiteral expectedResult = new IntLiteral(2018, Type.INT); Assert.assertEquals(expectedResult, actualResult); - actualResult = FEFunctions.year(new StringLiteral("1970-01-02 11:46:40")); - expectedResult = new IntLiteral(1970); + actualResult = FEFunctions.year(new DateLiteral("1970-01-02 11:46:40", Type.DATETIME)); + expectedResult = new IntLiteral(1970, Type.INT); Assert.assertEquals(expectedResult, actualResult); } @Test public void monthTest() throws AnalysisException { - IntLiteral actualResult = FEFunctions.month(new StringLiteral("2018-08-08")); - IntLiteral expectedResult = new IntLiteral(8); + IntLiteral actualResult = FEFunctions.month(new DateLiteral("2018-08-08", Type.DATE)); + IntLiteral expectedResult = new IntLiteral(8, Type.INT); Assert.assertEquals(expectedResult, actualResult); - actualResult = FEFunctions.month(new StringLiteral("1970-01-02 11:46:40")); - expectedResult = new IntLiteral(1); + actualResult = FEFunctions.month(new DateLiteral("1970-01-02 11:46:40", Type.DATETIME)); + expectedResult = new IntLiteral(1, Type.INT); Assert.assertEquals(expectedResult, actualResult); } @Test public void dayTest() throws AnalysisException { - IntLiteral actualResult = FEFunctions.day(new StringLiteral("2018-08-08")); - IntLiteral expectedResult = new IntLiteral(8); + IntLiteral actualResult = FEFunctions.day(new DateLiteral("2018-08-08", Type.DATE)); + IntLiteral expectedResult = new IntLiteral(8, Type.INT); Assert.assertEquals(expectedResult, actualResult); - actualResult = FEFunctions.day(new StringLiteral("1970-01-02 11:46:40")); - expectedResult = new IntLiteral(2); + actualResult = FEFunctions.day(new DateLiteral("1970-01-02 11:46:40", Type.DATETIME)); + expectedResult = new IntLiteral(2, Type.INT); Assert.assertEquals(expectedResult, actualResult); } diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index cc90080dddc50d..dba0d5b28d27a4 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -407,6 +407,7 @@ struct TMasterOpRequest { 6: optional i64 execMemLimit 7: optional i32 queryTimeout 8: optional string user_ip + 9: optional string time_zone } struct TColumnDefinition {