diff --git a/core/src/main/java/org/opensearch/sql/analysis/ExpressionAnalyzer.java b/core/src/main/java/org/opensearch/sql/analysis/ExpressionAnalyzer.java index d002e3672d0..013ee75829a 100644 --- a/core/src/main/java/org/opensearch/sql/analysis/ExpressionAnalyzer.java +++ b/core/src/main/java/org/opensearch/sql/analysis/ExpressionAnalyzer.java @@ -17,6 +17,7 @@ import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; import lombok.Getter; import org.opensearch.sql.analysis.symbol.Namespace; @@ -53,6 +54,7 @@ import org.opensearch.sql.ast.expression.subquery.ExistsSubquery; import org.opensearch.sql.ast.expression.subquery.InSubquery; import org.opensearch.sql.ast.expression.subquery.ScalarSubquery; +import org.opensearch.sql.calcite.utils.CalciteUtils; import org.opensearch.sql.data.model.ExprValueUtils; import org.opensearch.sql.data.type.ExprCoreType; import org.opensearch.sql.data.type.ExprType; @@ -191,6 +193,7 @@ public Expression visitRelevanceFieldList(RelevanceFieldList node, AnalysisConte @Override public Expression visitFunction(Function node, AnalysisContext context) { FunctionName functionName = FunctionName.of(node.getFuncName()); + validateCalciteOnlyFunction(functionName); List arguments = node.getFuncArgs().stream() .map( @@ -208,6 +211,34 @@ public Expression visitFunction(Function node, AnalysisContext context) { repository.compile(context.getFunctionProperties(), functionName, arguments); } + /** + * Validates that functions requiring Calcite engine are not used without it. + * + * @param functionName The function name to validate + */ + private void validateCalciteOnlyFunction(FunctionName functionName) { + if (isCalciteOnlyFunction(functionName)) { + throw CalciteUtils.getOnlyForCalciteException(functionName.getFunctionName().toUpperCase()); + } + } + + /** + * Checks if a function requires Calcite engine to be enabled. + * + * @param functionName The function name to check + * @return true if the function requires Calcite, false otherwise + */ + private boolean isCalciteOnlyFunction(FunctionName functionName) { + // Set of functions that are only supported with Calcite engine + Set calciteOnlyFunctions = + ImmutableSet.of( + BuiltinFunctionName.REGEX_MATCH.getName().getFunctionName(), + BuiltinFunctionName.STRFTIME.getName().getFunctionName()); + + return calciteOnlyFunctions.stream() + .anyMatch(f -> f.equalsIgnoreCase(functionName.getFunctionName())); + } + @SuppressWarnings("unchecked") @Override public Expression visitWindowFunction(WindowFunction node, AnalysisContext context) { diff --git a/core/src/main/java/org/opensearch/sql/expression/datetime/StrftimeFormatterUtil.java b/core/src/main/java/org/opensearch/sql/expression/datetime/StrftimeFormatterUtil.java new file mode 100644 index 00000000000..f42d376f649 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/expression/datetime/StrftimeFormatterUtil.java @@ -0,0 +1,252 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.expression.datetime; + +import com.google.common.collect.ImmutableMap; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.time.format.TextStyle; +import java.time.temporal.IsoFields; +import java.time.temporal.WeekFields; +import java.util.Locale; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.opensearch.sql.data.model.ExprStringValue; +import org.opensearch.sql.data.model.ExprValue; + +/** Utility class for POSIX-style strftime formatting. */ +public class StrftimeFormatterUtil { + + // Constants + private static final String PERCENT_PLACEHOLDER = "\u0001PERCENT\u0001"; + private static final String PERCENT_LITERAL = "%%"; + private static final int DEFAULT_NANOSECOND_PRECISION = 9; + private static final int DEFAULT_MILLISECOND_PRECISION = 3; + private static final int MICROSECOND_PRECISION = 6; + private static final long NANOS_PER_SECOND = 1_000_000_000L; + private static final long MILLIS_PER_SECOND = 1000L; + private static final long MAX_UNIX_TIMESTAMP = 32536771199L; + private static final int UNIX_TIMESTAMP_DIGITS = 10; + + // Pattern to match %N and %Q with optional precision digit + private static final Pattern SUBSECOND_PATTERN = Pattern.compile("%(\\d)?([NQ])"); + + @FunctionalInterface + private interface StrftimeFormatHandler { + String format(ZonedDateTime dateTime); + } + + private static final Map STRFTIME_HANDLERS = buildHandlers(); + + private static Map buildHandlers() { + return ImmutableMap.builder() + // Date and time combinations + .put( + "%c", + dt -> dt.format(DateTimeFormatter.ofPattern("EEE MMM dd HH:mm:ss yyyy", Locale.ROOT))) + .put( + "%+", + dt -> + dt.format(DateTimeFormatter.ofPattern("EEE MMM dd HH:mm:ss zzz yyyy", Locale.ROOT))) + + // Time formats + .put("%Ez", StrftimeFormatterUtil::formatTimezoneOffsetMinutes) + .put("%f", dt -> String.format("%06d", dt.getNano() / 1000)) + .put("%H", dt -> dt.format(DateTimeFormatter.ofPattern("HH", Locale.ROOT))) + .put("%I", dt -> dt.format(DateTimeFormatter.ofPattern("hh", Locale.ROOT))) + .put("%k", dt -> String.format("%2d", dt.getHour())) + .put("%M", dt -> dt.format(DateTimeFormatter.ofPattern("mm", Locale.ROOT))) + .put("%p", dt -> dt.format(DateTimeFormatter.ofPattern("a", Locale.ROOT))) + .put("%S", dt -> dt.format(DateTimeFormatter.ofPattern("ss", Locale.ROOT))) + .put("%s", dt -> String.valueOf(dt.toEpochSecond())) + .put("%T", dt -> dt.format(DateTimeFormatter.ofPattern("HH:mm:ss", Locale.ROOT))) + .put("%X", dt -> dt.format(DateTimeFormatter.ofPattern("HH:mm:ss", Locale.ROOT))) + + // Timezone formats + .put("%Z", dt -> dt.getZone().getDisplayName(TextStyle.SHORT, Locale.ROOT)) + .put("%z", dt -> dt.format(DateTimeFormatter.ofPattern("xx", Locale.ROOT))) + .put("%:z", dt -> dt.format(DateTimeFormatter.ofPattern("xxx", Locale.ROOT))) + .put("%::z", dt -> dt.format(DateTimeFormatter.ofPattern("xxx:ss", Locale.ROOT))) + .put("%:::z", dt -> dt.format(DateTimeFormatter.ofPattern("x", Locale.ROOT))) + + // Date formats + .put("%F", dt -> dt.format(DateTimeFormatter.ofPattern("yyyy-MM-dd", Locale.ROOT))) + .put("%x", dt -> dt.format(DateTimeFormatter.ofPattern("MM/dd/yyyy", Locale.ROOT))) + + // Weekday formats + .put("%A", dt -> dt.format(DateTimeFormatter.ofPattern("EEEE", Locale.ROOT))) + .put("%a", dt -> dt.format(DateTimeFormatter.ofPattern("EEE", Locale.ROOT))) + .put("%w", dt -> String.valueOf(dt.getDayOfWeek().getValue() % 7)) + + // Day formats + .put("%d", dt -> dt.format(DateTimeFormatter.ofPattern("dd", Locale.ROOT))) + .put("%e", dt -> String.format("%2d", dt.getDayOfMonth())) + .put("%j", dt -> dt.format(DateTimeFormatter.ofPattern("DDD", Locale.ROOT))) + + // Week formats + .put("%V", dt -> String.format("%02d", dt.get(IsoFields.WEEK_OF_WEEK_BASED_YEAR))) + .put("%U", dt -> String.format("%02d", dt.get(WeekFields.SUNDAY_START.weekOfYear()) - 1)) + + // Month formats + .put("%b", dt -> dt.format(DateTimeFormatter.ofPattern("MMM", Locale.ROOT))) + .put("%B", dt -> dt.format(DateTimeFormatter.ofPattern("MMMM", Locale.ROOT))) + .put("%m", dt -> dt.format(DateTimeFormatter.ofPattern("MM", Locale.ROOT))) + + // Year formats + .put("%C", dt -> String.format("%02d", dt.getYear() / 100)) + .put("%g", dt -> String.format("%02d", dt.get(IsoFields.WEEK_BASED_YEAR) % 100)) + .put("%G", dt -> String.format("%04d", dt.get(IsoFields.WEEK_BASED_YEAR))) + .put("%y", dt -> dt.format(DateTimeFormatter.ofPattern("yy", Locale.ROOT))) + .put("%Y", dt -> dt.format(DateTimeFormatter.ofPattern("yyyy", Locale.ROOT))) + + // Literal percent + .put(PERCENT_LITERAL, dt -> "%") + .build(); + } + + private StrftimeFormatterUtil() {} + + /** + * Format a ZonedDateTime using STRFTIME format specifiers. + * + * @param dateTime The ZonedDateTime to format + * @param formatString The STRFTIME format string + * @return Formatted string as ExprValue, or ExprNullValue if inputs are null + */ + public static ExprValue formatZonedDateTime(ZonedDateTime dateTime, String formatString) { + String result = processFormatString(formatString, dateTime); + return new ExprStringValue(result); + } + + /** Process the format string and replace all format specifiers. */ + private static String processFormatString(String formatString, ZonedDateTime dateTime) { + // Handle %N and %Q with precision first + String result = handleSubSecondFormats(formatString, dateTime); + + // Escape %% by replacing with placeholder + result = result.replace(PERCENT_LITERAL, PERCENT_PLACEHOLDER); + + // Replace all other format specifiers + result = replaceFormatSpecifiers(result, dateTime); + + // Restore literal percent signs + return result.replace(PERCENT_PLACEHOLDER, "%"); + } + + /** Replace all format specifiers in the string. */ + private static String replaceFormatSpecifiers(String input, ZonedDateTime dateTime) { + String result = input; + for (Map.Entry entry : STRFTIME_HANDLERS.entrySet()) { + String specifier = entry.getKey(); + if (result.contains(specifier)) { + String replacement = entry.getValue().format(dateTime); + result = result.replace(specifier, replacement); + } + } + return result; + } + + /** Handle %N and %Q subsecond formats with optional precision. */ + private static String handleSubSecondFormats(String format, ZonedDateTime dateTime) { + StringBuilder result = new StringBuilder(); + Matcher matcher = SUBSECOND_PATTERN.matcher(format); + + while (matcher.find()) { + String precisionStr = matcher.group(1); + String type = matcher.group(2); + + int precision = parsePrecision(precisionStr, type); + String replacement = formatSubseconds(dateTime, type, precision); + + matcher.appendReplacement(result, replacement); + } + matcher.appendTail(result); + + return result.toString(); + } + + /** Parse precision value for subsecond formats. */ + private static int parsePrecision(String precisionStr, String type) { + if (precisionStr != null) { + return Integer.parseInt(precisionStr); + } + // Default: %N=9 (nanoseconds), %Q=3 (milliseconds) + return "N".equals(type) ? DEFAULT_NANOSECOND_PRECISION : DEFAULT_MILLISECOND_PRECISION; + } + + /** Format subseconds based on type and precision. */ + private static String formatSubseconds(ZonedDateTime dateTime, String type, int precision) { + if ("N".equals(type)) { + // %N - subsecond digits (nanoseconds) + return formatNanoseconds(dateTime.getNano(), precision); + } else { + // %Q - subsecond component + return formatQSubseconds(dateTime, precision); + } + } + + /** Format nanoseconds with specified precision. */ + private static String formatNanoseconds(long nanos, int precision) { + double scaled = (double) nanos / NANOS_PER_SECOND; + long truncated = (long) (scaled * Math.pow(10, precision)); + return String.format("%0" + precision + "d", truncated); + } + + /** Format Q-type subseconds based on precision. */ + private static String formatQSubseconds(ZonedDateTime dateTime, int precision) { + switch (precision) { + case MICROSECOND_PRECISION: + // Microseconds + long micros = dateTime.getNano() / 1000; + return String.format("%06d", micros); + + case DEFAULT_NANOSECOND_PRECISION: + // Nanoseconds + return String.format("%09d", dateTime.getNano()); + + default: + // Default to milliseconds + long millis = dateTime.toInstant().toEpochMilli() % MILLIS_PER_SECOND; + return String.format("%03d", millis); + } + } + + /** Format timezone offset in minutes. */ + private static String formatTimezoneOffsetMinutes(ZonedDateTime dt) { + int offsetMinutes = dt.getOffset().getTotalSeconds() / 60; + return String.format("%+d", offsetMinutes); + } + + /** + * Extract the first 10 digits from a timestamp to get UNIX seconds. This handles timestamps that + * may be in milliseconds or nanoseconds. + * + * @param timestamp The timestamp value + * @return UNIX timestamp in seconds + */ + public static long extractUnixSeconds(double timestamp) { + // Return as-is if within valid Unix timestamp range + if (timestamp >= -MAX_UNIX_TIMESTAMP && timestamp <= MAX_UNIX_TIMESTAMP) { + return (long) timestamp; + } + + // For larger absolute values, extract first 10 digits (assumes milliseconds/nanoseconds) + return extractFirstNDigits(timestamp, UNIX_TIMESTAMP_DIGITS); + } + + /** Extract the first N digits from a number. */ + private static long extractFirstNDigits(double value, int digits) { + boolean isNegative = value < 0; + long absValue = Math.abs((long) value); + String valueStr = String.valueOf(absValue); + + long result = + valueStr.length() <= digits ? absValue : Long.parseLong(valueStr.substring(0, digits)); + + return isNegative ? -result : result; + } +} diff --git a/core/src/main/java/org/opensearch/sql/expression/function/BuiltinFunctionName.java b/core/src/main/java/org/opensearch/sql/expression/function/BuiltinFunctionName.java index 16fe2ea151c..ed65a472ca3 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/BuiltinFunctionName.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/BuiltinFunctionName.java @@ -125,6 +125,7 @@ public enum BuiltinFunctionName { UTC_TIME(FunctionName.of("utc_time")), UTC_TIMESTAMP(FunctionName.of("utc_timestamp")), UNIX_TIMESTAMP(FunctionName.of("unix_timestamp")), + STRFTIME(FunctionName.of("strftime")), WEEK(FunctionName.of("week")), WEEKDAY(FunctionName.of("weekday")), WEEKOFYEAR(FunctionName.of("weekofyear")), diff --git a/core/src/main/java/org/opensearch/sql/expression/function/PPLBuiltinOperators.java b/core/src/main/java/org/opensearch/sql/expression/function/PPLBuiltinOperators.java index be6f95b7a5e..055fedaec9d 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/PPLBuiltinOperators.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/PPLBuiltinOperators.java @@ -76,6 +76,7 @@ import org.opensearch.sql.expression.function.udf.datetime.LastDayFunction; import org.opensearch.sql.expression.function.udf.datetime.PeriodNameFunction; import org.opensearch.sql.expression.function.udf.datetime.SecToTimeFunction; +import org.opensearch.sql.expression.function.udf.datetime.StrftimeFunction; import org.opensearch.sql.expression.function.udf.datetime.SysdateFunction; import org.opensearch.sql.expression.function.udf.datetime.TimestampAddFunction; import org.opensearch.sql.expression.function.udf.datetime.TimestampDiffFunction; @@ -170,6 +171,7 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { public static final SqlOperator WEEKDAY = new WeekdayFunction().toUDF("WEEKDAY"); public static final SqlOperator UNIX_TIMESTAMP = new UnixTimestampFunction().toUDF("UNIX_TIMESTAMP"); + public static final SqlOperator STRFTIME = new StrftimeFunction().toUDF("STRFTIME"); public static final SqlOperator TO_SECONDS = new ToSecondsFunction().toUDF("TO_SECONDS"); public static final SqlOperator ADDTIME = adaptExprMethodWithPropertiesToUDF( diff --git a/core/src/main/java/org/opensearch/sql/expression/function/PPLFuncImpTable.java b/core/src/main/java/org/opensearch/sql/expression/function/PPLFuncImpTable.java index 59ff2866bee..831ed346a5a 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/PPLFuncImpTable.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/PPLFuncImpTable.java @@ -188,6 +188,7 @@ import static org.opensearch.sql.expression.function.BuiltinFunctionName.STDDEV_POP; import static org.opensearch.sql.expression.function.BuiltinFunctionName.STDDEV_SAMP; import static org.opensearch.sql.expression.function.BuiltinFunctionName.STRCMP; +import static org.opensearch.sql.expression.function.BuiltinFunctionName.STRFTIME; import static org.opensearch.sql.expression.function.BuiltinFunctionName.STR_TO_DATE; import static org.opensearch.sql.expression.function.BuiltinFunctionName.SUBDATE; import static org.opensearch.sql.expression.function.BuiltinFunctionName.SUBSTR; @@ -755,6 +756,7 @@ void populate() { registerOperator(YEARWEEK, PPLBuiltinOperators.YEARWEEK); registerOperator(WEEKDAY, PPLBuiltinOperators.WEEKDAY); registerOperator(UNIX_TIMESTAMP, PPLBuiltinOperators.UNIX_TIMESTAMP); + registerOperator(STRFTIME, PPLBuiltinOperators.STRFTIME); registerOperator(TO_SECONDS, PPLBuiltinOperators.TO_SECONDS); registerOperator(TO_DAYS, PPLBuiltinOperators.TO_DAYS); registerOperator(ADDTIME, PPLBuiltinOperators.ADDTIME); diff --git a/core/src/main/java/org/opensearch/sql/expression/function/udf/datetime/StrftimeFunction.java b/core/src/main/java/org/opensearch/sql/expression/function/udf/datetime/StrftimeFunction.java new file mode 100644 index 00000000000..653455609d4 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/expression/function/udf/datetime/StrftimeFunction.java @@ -0,0 +1,185 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.expression.function.udf.datetime; + +import static org.opensearch.sql.calcite.utils.UserDefinedFunctionUtils.convertToExprValues; + +import java.time.Instant; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.util.List; +import org.apache.calcite.adapter.enumerable.NotNullImplementor; +import org.apache.calcite.adapter.enumerable.NullPolicy; +import org.apache.calcite.adapter.enumerable.RexToLixTranslator; +import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.sql.type.*; +import org.opensearch.sql.data.model.ExprDateValue; +import org.opensearch.sql.data.model.ExprDoubleValue; +import org.opensearch.sql.data.model.ExprFloatValue; +import org.opensearch.sql.data.model.ExprIntegerValue; +import org.opensearch.sql.data.model.ExprLongValue; +import org.opensearch.sql.data.model.ExprTimestampValue; +import org.opensearch.sql.data.model.ExprValue; +import org.opensearch.sql.expression.datetime.StrftimeFormatterUtil; +import org.opensearch.sql.expression.function.ImplementorUDF; +import org.opensearch.sql.expression.function.UDFOperandMetadata; + +/** + * Implementation of the STRFTIME function. This function takes a UNIX timestamp (in seconds) and + * formats it according to POSIX-style format specifiers. + * + *

All timestamps are interpreted as UTC and all text formatting uses Locale.ROOT for consistent, + * language-neutral output regardless of server locale settings. + */ +public class StrftimeFunction extends ImplementorUDF { + + // Maximum valid UNIX timestamp per MySQL documentation + // On 64-bit platforms, effective maximum is 32536771199.999999 (3001-01-18 23:59:59.999999 UTC) + private static final long MAX_UNIX_TIMESTAMP = 32536771199L; + private static final int NANOS_PER_SECOND = 1_000_000_000; + + public StrftimeFunction() { + super(new StrftimeImplementor(), NullPolicy.STRICT); + } + + @Override + public SqlReturnTypeInference getReturnTypeInference() { + return ReturnTypes.VARCHAR; + } + + @Override + public UDFOperandMetadata getOperandMetadata() { + // Accepts (NUMERIC|TIMESTAMP, STRING) -> STRING + // Note: STRING is NOT accepted for first parameter - use unix_timestamp() to convert + // Calcite will auto-cast DATE and TIME to TIMESTAMP + return UDFOperandMetadata.wrap( + (CompositeOperandTypeChecker) + OperandTypes.family(SqlTypeFamily.NUMERIC, SqlTypeFamily.CHARACTER) + .or(OperandTypes.family(SqlTypeFamily.TIMESTAMP, SqlTypeFamily.CHARACTER))); + } + + public static class StrftimeImplementor implements NotNullImplementor { + @Override + public Expression implement( + RexToLixTranslator rexToLixTranslator, RexCall rexCall, List list) { + List operands = convertToExprValues(list, rexCall); + return Expressions.call(StrftimeFunction.class, "strftime", operands); + } + } + + /** + * Static method for Calcite implementation. + * + * @param unixTime The UNIX timestamp to format + * @param formatString The format string with POSIX-style specifiers + * @return Formatted date string or null if invalid input + */ + public static String strftime(ExprValue unixTime, ExprValue formatString) { + if (unixTime.isNull() || formatString.isNull()) { + return null; + } + Double unixSecondsDouble = extractUnixSecondsWithFraction(unixTime); + // Combine null checks and validation + if (unixSecondsDouble == null || !isValidTimestamp(unixSecondsDouble.longValue())) { + return null; + } + ZonedDateTime dateTime = convertToZonedDateTimeWithFraction(unixSecondsDouble); + return formatDateTime(dateTime, formatString.stringValue()); + } + + /** + * Extract UNIX seconds with fractional part from various input types. + * + * @param unixTime The input value + * @return UNIX seconds with fraction or null if cannot extract + */ + private static Double extractUnixSecondsWithFraction(ExprValue unixTime) { + // 1. Handle TIMESTAMP types (from now(), from_unixtime(), timestamp(), etc.) + if (unixTime instanceof ExprTimestampValue timestamp) { + Instant instant = timestamp.timestampValue(); + return instant.getEpochSecond() + instant.getNano() / (double) NANOS_PER_SECOND; + } + + // 2. Handle DATE types (convert to timestamp at midnight UTC) + if (unixTime instanceof ExprDateValue date) { + // DATE converts to timestamp at midnight UTC + Instant instant = date.timestampValue(); + return instant.getEpochSecond() + instant.getNano() / (double) NANOS_PER_SECOND; + } + + // 3. Handle numeric types (UNIX timestamps in seconds or milliseconds) + Double numericValue = extractNumericValue(unixTime); + if (numericValue == null) { + return null; // Not a numeric type (could be string, etc.) + } + + // Auto-detect if value is in milliseconds (>= 100 billion) + double absValue = Math.abs(numericValue); + if (absValue >= 1e11) { // >= 100 billion, likely milliseconds + return numericValue / 1000.0; + } else { + return numericValue; + } + } + + /** + * Extract numeric value from various numeric ExprValue types. + * + * @param value The input ExprValue + * @return Double value or null if not a numeric type + */ + private static Double extractNumericValue(ExprValue value) { + if (value instanceof ExprDoubleValue) { + return value.doubleValue(); + } else if (value instanceof ExprLongValue) { + return (double) value.longValue(); + } else if (value instanceof ExprIntegerValue) { + return (double) value.integerValue(); + } else if (value instanceof ExprFloatValue) { + return (double) value.floatValue(); + } + + // Not a numeric type + return null; + } + + /** + * Check if the timestamp is within valid range. Accepts negative values for dates before 1970. + * + * @param unixSeconds The UNIX timestamp in seconds + * @return true if valid, false otherwise + */ + private static boolean isValidTimestamp(long unixSeconds) { + return unixSeconds >= -MAX_UNIX_TIMESTAMP && unixSeconds <= MAX_UNIX_TIMESTAMP; + } + + /** + * Convert UNIX seconds with fractional part to ZonedDateTime in UTC. + * + * @param unixSeconds The UNIX timestamp in seconds with fractional seconds + * @return ZonedDateTime in UTC + */ + private static ZonedDateTime convertToZonedDateTimeWithFraction(double unixSeconds) { + long seconds = (long) unixSeconds; + int nanos = (int) ((unixSeconds - seconds) * NANOS_PER_SECOND); + Instant instant = Instant.ofEpochSecond(seconds, nanos); + return ZonedDateTime.ofInstant(instant, ZoneId.of("UTC")); + } + + /** + * Format the datetime using the format string. + * + * @param dateTime The datetime to format + * @param format The format string + * @return Formatted string or null if formatting fails + */ + private static String formatDateTime(ZonedDateTime dateTime, String format) { + ExprValue result = StrftimeFormatterUtil.formatZonedDateTime(dateTime, format); + return result.stringValue(); + } +} diff --git a/core/src/test/java/org/opensearch/sql/expression/datetime/StrftimeFunctionTest.java b/core/src/test/java/org/opensearch/sql/expression/datetime/StrftimeFunctionTest.java new file mode 100644 index 00000000000..0daeb31bef7 --- /dev/null +++ b/core/src/test/java/org/opensearch/sql/expression/datetime/StrftimeFunctionTest.java @@ -0,0 +1,535 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.sql.expression.datetime; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; + +import java.time.Instant; +import java.time.LocalDate; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import org.junit.jupiter.api.Test; +import org.opensearch.sql.data.model.ExprDateValue; +import org.opensearch.sql.data.model.ExprDoubleValue; +import org.opensearch.sql.data.model.ExprLongValue; +import org.opensearch.sql.data.model.ExprNullValue; +import org.opensearch.sql.data.model.ExprStringValue; +import org.opensearch.sql.data.model.ExprTimestampValue; +import org.opensearch.sql.data.model.ExprValue; +import org.opensearch.sql.data.type.ExprCoreType; +import org.opensearch.sql.data.type.ExprType; +import org.opensearch.sql.expression.function.udf.datetime.StrftimeFunction; + +/** + * Unit tests for STRFTIME function implementation. Tests the core functionality without going + * through the full expression system. All tests use Locale.ROOT for consistent results. + */ +public class StrftimeFunctionTest { + @Test + public void testStrftimeWithBasicFormat() { + // Unix timestamp for 2018-03-19T13:55:03 UTC + long unixTime = 1521467703L; + String result = + StrftimeFunction.strftime( + new ExprDoubleValue(unixTime), new ExprStringValue("%Y-%m-%dT%H:%M:%S")); + assertEquals("2018-03-19T13:55:03", result); + } + + @Test + public void testStrftimeWithISOFormat() { + long unixTime = 1521467703L; + String result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%F %T")); + assertEquals("2018-03-19 13:55:03", result); + } + + @Test + public void testStrftimeWithWeekdayAndMonth() { + long unixTime = 1521467703L; + String result = + StrftimeFunction.strftime( + new ExprDoubleValue(unixTime), new ExprStringValue("%a %b %d, %Y")); + assertEquals("Mon Mar 19, 2018", result); + } + + @Test + public void testStrftimeWithFullWeekdayAndMonth() { + long unixTime = 1521467703L; + String result = + StrftimeFunction.strftime( + new ExprDoubleValue(unixTime), new ExprStringValue("%A %B %d, %Y")); + assertEquals("Mon Mar 19, 2018", result); + } + + @Test + public void testStrftimeWith12HourFormat() { + long unixTime = 1521467703L; + String result = + StrftimeFunction.strftime( + new ExprDoubleValue(unixTime), new ExprStringValue("%I:%M:%S %p")); + assertEquals("01:55:03 PM", result); + } + + @Test + public void testStrftimeWithEpochSeconds() { + long unixTime = 1521467703L; + String result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%s")); + assertEquals("1521467703", result); + } + + @Test + public void testStrftimeWithLongTimestamp() { + // Test with millisecond timestamp (auto-detected and converted to seconds) + long unixTime = 1521467703123L; // milliseconds + String result = + StrftimeFunction.strftime( + new ExprLongValue(unixTime), new ExprStringValue("%Y-%m-%dT%H:%M:%S")); + assertEquals("2018-03-19T13:55:03", result); + } + + @Test + public void testStrftimeWithSubsecondPrecision() { + long unixTime = 1521467703L; + String result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%S.%3Q")); + assertEquals("03.000", result); + result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%S.%6Q")); + assertEquals("03.000000", result); + result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%S.%9Q")); + assertEquals("03.000000000", result); + result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%S.%5Q")); + assertEquals("03.000", result); + result = StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%S.%Q")); + assertEquals("03.000", result); + } + + @Test + public void testStrftimeWithNanosecondPrecision() { + // Test with fractional seconds (using value with limited precision to avoid floating point + // issues) + double unixTime = 1521467703.125; // Use simple fraction to avoid precision issues + String result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%S.%N")); + assertEquals("03.125000000", result); + result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%S.%3N")); + assertEquals("03.125", result); + result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%S.%6N")); + assertEquals("03.125000", result); + result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%S.%9N")); + assertEquals("03.125000000", result); + result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%S.%5N")); + assertEquals("03.12500", result); + } + + @Test + public void testStrftimeWithMicroseconds() { + long unixTime = 1521467703L; + String result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%f")); + assertEquals("000000", result); + } + + @Test + public void testStrftimeWithWeekNumbers() { + long unixTime = 1521467703L; + String result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%V %U %w")); + assertEquals("12 11 1", result); + } + + @Test + public void testStrftimeWithDayOfYear() { + long unixTime = 1521467703L; + String result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%j")); + assertEquals("078", result); + } + + @Test + public void testStrftimeWithCentury() { + long unixTime = 1521467703L; + String result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%C")); + assertEquals("20", result); + } + + @Test + public void testStrftimeWithYearFormats() { + long unixTime = 1521467703L; + String result = + StrftimeFunction.strftime( + new ExprDoubleValue(unixTime), new ExprStringValue("%Y %y %G %g")); + assertEquals("2018 18 2018 18", result); + } + + @Test + public void testStrftimeWithSpacePaddedFormats() { + long unixTime = 1517472303L; // 2018-02-01 08:05:03 + String result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%e %k")); + assertEquals(" 1 8", result); + } + + @Test + public void testStrftimeWithPercentLiteral() { + long unixTime = 1521467703L; + // Test literal percent: %% should become % + String result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%%")); + assertEquals("%", result); + // %%Y should become %Y (not expanding Y) + result = StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%%Y")); + assertEquals("%Y", result); + } + + @Test + public void testStrftimeWithStringTimestamp() { + // String inputs are not supported - should return null + String result = + StrftimeFunction.strftime( + new ExprStringValue("1521467703"), new ExprStringValue("%Y-%m-%d")); + assertNull(result); + } + + @Test + public void testStrftimeWithLongValue() { + String result = + StrftimeFunction.strftime(new ExprLongValue(1521467703L), new ExprStringValue("%Y-%m-%d")); + assertEquals("2018-03-19", result); + } + + @Test + public void testStrftimeWithTimestampValue() { + // Create timestamp using Instant for proper epoch second conversion + Instant instant = Instant.ofEpochSecond(1521467703L); // 2018-03-19T13:55:03 UTC + ExprTimestampValue timestamp = new ExprTimestampValue(instant); + String result = StrftimeFunction.strftime(timestamp, new ExprStringValue("%Y-%m-%d %H:%M:%S")); + assertEquals("2018-03-19 13:55:03", result); + } + + @Test + public void testStrftimeWithDateValue() { + // Create a DATE value - should convert to timestamp at midnight UTC + LocalDate date = LocalDate.of(2020, 9, 16); + ExprDateValue dateValue = new ExprDateValue(date); + String result = StrftimeFunction.strftime(dateValue, new ExprStringValue("%Y-%m-%d")); + assertEquals("2020-09-16", result); + + // Also verify time is at midnight + result = StrftimeFunction.strftime(dateValue, new ExprStringValue("%Y-%m-%d %H:%M:%S")); + assertEquals("2020-09-16 00:00:00", result); + } + + @Test + public void testStrftimeWithNullTimestamp() { + String result = StrftimeFunction.strftime(ExprNullValue.of(), new ExprStringValue("%Y-%m-%d")); + assertNull(result); + } + + @Test + public void testStrftimeWithNullFormat() { + String result = StrftimeFunction.strftime(new ExprDoubleValue(1521467703L), ExprNullValue.of()); + assertNull(result); + } + + @Test + public void testStrftimeWithNegativeTimestamp() { + // Negative timestamps represent dates before 1970 + // -1 represents 1969-12-31 23:59:59 UTC + String result = + StrftimeFunction.strftime(new ExprDoubleValue(-1), new ExprStringValue("%Y-%m-%d")); + assertEquals("1969-12-31", result); + + // Test another negative timestamp: -86400 = 1969-12-31 00:00:00 UTC + result = + StrftimeFunction.strftime( + new ExprDoubleValue(-86400), new ExprStringValue("%Y-%m-%d %H:%M:%S")); + assertEquals("1969-12-31 00:00:00", result); + } + + @Test + public void testStrftimeWithOutOfRangeTimestamp() { + // 32536771200 is above the max valid range (32536771199) + // This should return null because it's out of the valid FROM_UNIXTIME range + String result = + StrftimeFunction.strftime( + new ExprDoubleValue(32536771200.0), // Above max range by 1 second + new ExprStringValue("%Y-%m-%d")); + assertNull(result); + } + + @Test + public void testStrftimeWithInvalidStringTimestamp() { + String result = + StrftimeFunction.strftime(new ExprStringValue("invalid"), new ExprStringValue("%Y-%m-%d")); + assertNull(result); + } + + @Test + public void testStrftimeFormatterUtilExtractUnixSeconds() { + // Test extracting first 10 digits + assertEquals(1521467703L, StrftimeFormatterUtil.extractUnixSeconds(1521467703049000000.0)); + assertEquals(1521467703L, StrftimeFormatterUtil.extractUnixSeconds(1521467703.0)); + assertEquals(152146770L, StrftimeFormatterUtil.extractUnixSeconds(152146770.0)); + } + + @Test + public void testStrftimeWithComplexFormat() { + long unixTime = 1521467703L; + String result = + StrftimeFunction.strftime( + new ExprDoubleValue(unixTime), new ExprStringValue("Date: %F, Time: %T, Weekday: %A")); + assertEquals("Date: 2018-03-19, Time: 13:55:03, Weekday: Mon", result); + } + + @Test + public void testStrftimeWithLocaleFormats() { + long unixTime = 1521467703L; + String result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%c")); + assertEquals("Mon Mar 19 13:55:03 2018", result); + result = StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%+")); + assertEquals("Mon Mar 19 13:55:03 UTC 2018", result); + } + + @Test + public void testStrftimeWithDateLocaleFormat() { + long unixTime = 1521467703L; + String result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%x")); + assertEquals("03/19/2018", result); + } + + @Test + public void testStrftimeWithTimeLocaleFormat() { + long unixTime = 1521467703L; + String result = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%X")); + assertEquals("13:55:03", result); + } + + @Test + public void testStrftimeWithTimezoneFormats() { + long unixTime = 1521467703L; + // Test %Ez - timezone offset in minutes + String ezResult = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%Ez")); + assertEquals("+0", ezResult); + // Test %z - timezone offset +hhmm + String zResult = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%z")); + assertEquals("+0000", zResult); + // Test %Z - timezone abbreviation + String bigZResult = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%Z")); + assertEquals("UTC", bigZResult); + // Test %:z - timezone offset with colon + String colonZResult = + StrftimeFunction.strftime(new ExprDoubleValue(unixTime), new ExprStringValue("%:z")); + assertEquals("+00:00", colonZResult); + } + + @Test + public void testStrftimeFormatterDirectly() { + // Test the formatter utility directly + Instant instant = Instant.ofEpochSecond(1521467703L); + ZonedDateTime dateTime = ZonedDateTime.ofInstant(instant, ZoneId.of("UTC")); + ExprValue result = StrftimeFormatterUtil.formatZonedDateTime(dateTime, "%Y-%m-%d"); + assertEquals("2018-03-19", result.stringValue()); + } + + @Test + public void testStrftimeFunctionConstructor() { + // Test the constructor and basic setup + StrftimeFunction function = new StrftimeFunction(); + assertNotNull(function); + assertNotNull(function.getReturnTypeInference()); + assertNotNull(function.getOperandMetadata()); + } + + @Test + public void testExtractNumericValueWithNonStringExprValue() { + // Test with a custom ExprValue that doesn't match our supported types + // Since we removed string support, custom ExprValue types that aren't + // ExprDoubleValue, ExprLongValue, ExprIntegerValue, or ExprFloatValue + // will return null + ExprValue customValue = + new ExprValue() { + @Override + public Object value() { + return 1521467703; + } + + @Override + public ExprType type() { + return ExprCoreType.INTEGER; + } + + @Override + public String toString() { + return "1521467703"; + } + + @Override + public int compareTo(ExprValue other) { + return 0; + } + + @Override + public Object valueForCalcite() { + return value(); + } + }; + + String result = StrftimeFunction.strftime(customValue, new ExprStringValue("%Y-%m-%d")); + // Custom ExprValue implementations that aren't one of our supported types return null + assertNull(result); + } + + @Test + public void testIsValidTimestampBoundaries() { + // Test boundary conditions for isValidTimestamp + + // Test exactly at max boundary + String result = + StrftimeFunction.strftime( + new ExprDoubleValue(32536771199L), new ExprStringValue("%Y-%m-%d")); + assertEquals("3001-01-18", result); + + // Test exactly at 0 (epoch) + result = + StrftimeFunction.strftime(new ExprDoubleValue(0), new ExprStringValue("%Y-%m-%d %H:%M:%S")); + assertEquals("1970-01-01 00:00:00", result); + } + + @Test + public void testExtractNumericValueWithInvalidDouble() { + // Test parsing failure in extractNumericValue + ExprValue customValue = + new ExprValue() { + @Override + public Object value() { + return "not_a_number"; + } + + @Override + public ExprType type() { + return ExprCoreType.STRING; + } + + @Override + public String toString() { + return "not_a_number"; + } + + @Override + public int compareTo(ExprValue other) { + return 0; + } + + @Override + public Object valueForCalcite() { + return value(); + } + }; + + String result = StrftimeFunction.strftime(customValue, new ExprStringValue("%Y-%m-%d")); + assertNull(result); + } + + @Test + public void testExtractFirstNDigitsWithShortNumber() { + // Test extractFirstNDigits with a number shorter than requested digits + // This covers the branch where valueStr.length() <= digits + assertEquals(123L, StrftimeFormatterUtil.extractUnixSeconds(123.0)); + + // Test with exactly 10 digits + assertEquals(1234567890L, StrftimeFormatterUtil.extractUnixSeconds(1234567890.0)); + } + + @Test + public void testExtractFirstNDigitsWithLongNumber() { + // Test extractFirstNDigits with a number longer than 10 digits + // This covers the branch where substring is used + assertEquals(1234567890L, StrftimeFormatterUtil.extractUnixSeconds(12345678901234.0)); + assertEquals(1521467703L, StrftimeFormatterUtil.extractUnixSeconds(1521467703123456789.0)); + } + + @Test + public void testStrftimeWithBoundaryTimestamp() { + // Test with timestamp at the upper boundary (32536771199) + long maxTimestamp = 32536771199L; + String result = + StrftimeFunction.strftime( + new ExprDoubleValue(maxTimestamp), new ExprStringValue("%Y-%m-%d %H:%M:%S")); + assertEquals("3001-01-18 23:59:59", result); + + // Test with timestamp just above the boundary (should return null) + result = + StrftimeFunction.strftime( + new ExprDoubleValue(maxTimestamp + 1), new ExprStringValue("%Y-%m-%d")); + assertNull(result); + } + + @Test + public void testStrftimeWithLargeValueAsMilliseconds() { + // Test with value > MAX_UNIX_TIMESTAMP that should be interpreted as milliseconds + // Note: values < 100 billion are treated as invalid seconds, not milliseconds + // This avoids treating values like 32536771200 (just above max) as milliseconds + + // Test with current time in milliseconds (around 1.7 trillion) + // 1600000000000 ms = 1600000000 seconds = 2020-09-13 + String result = + StrftimeFunction.strftime( + new ExprDoubleValue(1600000000000L), new ExprStringValue("%Y-%m-%d")); + assertEquals("2020-09-13", result); + + // Test with negative value that's within seconds range + // -86400000 is treated as seconds (not milliseconds) since it's within valid range + // -86400000 seconds = 1967-04-07 + result = + StrftimeFunction.strftime(new ExprDoubleValue(-86400000L), new ExprStringValue("%Y-%m-%d")); + assertEquals("1967-04-07", result); + } + + @Test + public void testStrftimeWithValueTooLargeEvenAsMilliseconds() { + // Test with value so large that even as milliseconds it exceeds the range + // 32536771199999 ms would be 32536771199.999 seconds (beyond max) + String result = + StrftimeFunction.strftime( + new ExprDoubleValue(32536771200000L), // Too large even as milliseconds + new ExprStringValue("%Y-%m-%d")); + assertNull(result); + } + + @Test + public void testStrftimeDeadZone() { + // Values between MAX_UNIX_TIMESTAMP and 100 billion are treated as invalid + // This prevents values like 32536771200 (just above max) from being misinterpreted + + // 50 billion - in the "dead zone", returns null + String result = + StrftimeFunction.strftime( + new ExprDoubleValue(50000000000L), new ExprStringValue("%Y-%m-%d")); + assertNull(result); + + // Just above MAX_UNIX_TIMESTAMP - should return null, not interpret as milliseconds + result = + StrftimeFunction.strftime( + new ExprDoubleValue(32536771200L), new ExprStringValue("%Y-%m-%d")); + assertNull(result); + } +} diff --git a/docs/user/ppl/functions/datetime.rst b/docs/user/ppl/functions/datetime.rst index 2c210d54a3a..bd69425a2dd 100644 --- a/docs/user/ppl/functions/datetime.rst +++ b/docs/user/ppl/functions/datetime.rst @@ -1473,6 +1473,172 @@ Example:: +------------------------------------+ +STRFTIME +-------- + +**Version: 3.3.0** + +Description +>>>>>>>>>>> + +Usage: strftime(time, format) takes a UNIX timestamp (in seconds) and renders it as a string using the format specified. For numeric inputs, the UNIX time must be in seconds. Values greater than 100000000000 are automatically treated as milliseconds and converted to seconds. +You can use time format variables with the strftime function. This function performs the reverse operation of `UNIX_TIMESTAMP`_ and is similar to `FROM_UNIXTIME`_ but with POSIX-style format specifiers. + +.. note:: + - **Available only when Calcite engine is enabled** + - All timestamps are interpreted as UTC timezone + - Text formatting uses language-neutral Locale.ROOT (weekday and month names appear in abbreviated form) + - String inputs are NOT supported - use `unix_timestamp()` to convert strings first + - Functions that return date/time values (like `date()`, `now()`, `timestamp()`) are supported + +Argument type: INTEGER/LONG/DOUBLE/TIMESTAMP, STRING + +Return type: STRING + +Format specifiers: + +.. list-table:: The following table describes the available specifier arguments. + :widths: 20 80 + :header-rows: 1 + + * - Specifier + - Description + * - %a + - Abbreviated weekday name (Mon..Sun) + * - %A + - Weekday name (Mon..Sun) - Note: Locale.ROOT uses abbreviated form + * - %b + - Abbreviated month name (Jan..Dec) + * - %B + - Month name (Jan..Dec) - Note: Locale.ROOT uses abbreviated form + * - %c + - Date and time (e.g., Mon Jul 18 09:30:00 2019) + * - %C + - Century as 2-digit decimal number + * - %d + - Day of the month, zero-padded (01..31) + * - %e + - Day of the month, space-padded ( 1..31) + * - %Ez + - Timezone offset in minutes from UTC (e.g., +0 for UTC, +330 for IST, -300 for EST) + * - %f + - Microseconds as decimal number (000000..999999) + * - %F + - ISO 8601 date format (%Y-%m-%d) + * - %g + - ISO 8601 year without century (00..99) + * - %G + - ISO 8601 year with century + * - %H + - Hour (24-hour clock) (00..23) + * - %I + - Hour (12-hour clock) (01..12) + * - %j + - Day of year (001..366) + * - %k + - Hour (24-hour clock), space-padded ( 0..23) + * - %m + - Month as decimal number (01..12) + * - %M + - Minute (00..59) + * - %N + - Subsecond digits (default %9N = nanoseconds). Accepts any precision value from 1-9 (e.g., %3N = 3 digits, %5N = 5 digits, %9N = 9 digits). The precision directly controls the number of digits displayed + * - %p + - AM or PM + * - %Q + - Subsecond component (default milliseconds). Can specify precision: %3Q = milliseconds, %6Q = microseconds, %9Q = nanoseconds. Other precision values (e.g., %5Q) default to %3Q + * - %s + - UNIX Epoch timestamp in seconds + * - %S + - Second (00..59) + * - %T + - Time in 24-hour notation (%H:%M:%S) + * - %U + - Week of year starting from 0 (00..53) + * - %V + - ISO week number (01..53) + * - %w + - Weekday as decimal (0=Sunday..6=Saturday) + * - %x + - Date in MM/dd/yyyy format (e.g., 07/13/2019) + * - %X + - Time in HH:mm:ss format (e.g., 09:30:00) + * - %y + - Year without century (00..99) + * - %Y + - Year with century + * - %z + - Timezone offset (+hhmm or -hhmm) + * - %:z + - Timezone offset with colon (+hh:mm or -hh:mm) + * - %::z + - Timezone offset with colons (+hh:mm:ss) + * - %:::z + - Timezone offset hour only (+hh or -hh) + * - %Z + - Timezone abbreviation (e.g., EST, PDT) + * - %% + - Literal % character + +Examples:: + + #os> source=people | eval `strftime(1521467703, "%Y-%m-%dT%H:%M:%S")` = strftime(1521467703, "%Y-%m-%dT%H:%M:%S") | fields `strftime(1521467703, "%Y-%m-%dT%H:%M:%S")` + fetched rows / total rows = 1/1 + +-------------------------------------------+ + | strftime(1521467703, "%Y-%m-%dT%H:%M:%S") | + |-------------------------------------------| + | 2018-03-19T13:55:03 | + +-------------------------------------------+ + + #os> source=people | eval `strftime(1521467703, "%F %T")` = strftime(1521467703, "%F %T") | fields `strftime(1521467703, "%F %T")` + fetched rows / total rows = 1/1 + +-------------------------------+ + | strftime(1521467703, "%F %T") | + |-------------------------------| + | 2018-03-19 13:55:03 | + +-------------------------------+ + + #os> source=people | eval `strftime(1521467703, "%a %b %d, %Y")` = strftime(1521467703, "%a %b %d, %Y") | fields `strftime(1521467703, "%a %b %d, %Y")` + fetched rows / total rows = 1/1 + +--------------------------------------+ + | strftime(1521467703, "%a %b %d, %Y") | + |--------------------------------------| + | Mon Mar 19, 2018 | + +--------------------------------------+ + + #os> source=people | eval `strftime(1521467703, "%%Y")` = strftime(1521467703, "%%Y") | fields `strftime(1521467703, "%%Y")` + fetched rows / total rows = 1/1 + +---------------------------+ + | strftime(1521467703, "%%Y") | + |---------------------------| + | %Y | + +---------------------------+ + + #os> source=people | eval `strftime(date('2020-09-16'), "%Y-%m-%d")` = strftime(date('2020-09-16'), "%Y-%m-%d") | fields `strftime(date('2020-09-16'), "%Y-%m-%d")` + fetched rows / total rows = 1/1 + +----------------------------------------+ + | strftime(date('2020-09-16'), "%Y-%m-%d") | + |-----------------------------------------| + | 2020-09-16 | + +----------------------------------------+ + + #os> source=people | eval `strftime(timestamp('2020-09-16 14:30:00'), "%F %T")` = strftime(timestamp('2020-09-16 14:30:00'), "%F %T") | fields `strftime(timestamp('2020-09-16 14:30:00'), "%F %T")` + fetched rows / total rows = 1/1 + +--------------------------------------------------+ + | strftime(timestamp('2020-09-16 14:30:00'), "%F %T") | + |---------------------------------------------------| + | 2020-09-16 14:30:00 | + +--------------------------------------------------+ + + #os> source=people | eval `strftime(now(), "%Y-%m-%d %H:%M:%S")` = strftime(now(), "%Y-%m-%d %H:%M:%S") | fields `strftime(now(), "%Y-%m-%d %H:%M:%S")` + fetched rows / total rows = 1/1 + +------------------------------------+ + | strftime(now(), "%Y-%m-%d %H:%M:%S") | + |-------------------------------------| + | 2025-09-03 12:30:45 | + +------------------------------------+ + + STR_TO_DATE ----------- diff --git a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteDateTimeFunctionIT.java b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteDateTimeFunctionIT.java index a069c6566ba..8eee5c01f7c 100644 --- a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteDateTimeFunctionIT.java +++ b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteDateTimeFunctionIT.java @@ -5,8 +5,17 @@ package org.opensearch.sql.calcite.remote; +import static org.opensearch.sql.legacy.TestsConstants.TEST_INDEX_DATE; +import static org.opensearch.sql.legacy.TestsConstants.TEST_INDEX_DATE_FORMATS; +import static org.opensearch.sql.util.MatcherUtils.rows; +import static org.opensearch.sql.util.MatcherUtils.verifyDataRows; + import java.io.IOException; +import java.time.LocalDate; +import java.time.ZoneOffset; +import org.json.JSONObject; import org.junit.Ignore; +import org.junit.Test; import org.opensearch.sql.ppl.DateTimeFunctionIT; public class CalciteDateTimeFunctionIT extends DateTimeFunctionIT { @@ -21,4 +30,179 @@ public void init() throws Exception { @Ignore @Override public void testUnixTimestampWithTimestampString() throws IOException {} + + @Test + public void testStrftimeWithComplexFormat() throws IOException { + // Test strftime with complex format string including various date/time components + JSONObject result = + executeQuery( + String.format( + "source=%s | eval result = strftime(1521467703, '%s') | fields result | head 1", + TEST_INDEX_DATE, "%a, %b %d, %Y %I:%M:%S %p %Z")); + verifyDataRows(result, rows("Mon, Mar 19, 2018 01:55:03 PM UTC")); + } + + @Test + public void testStrftimeWithVariousInputTypes() throws IOException { + // Test 1: Direct UNIX timestamp (INTEGER/LONG) + JSONObject result1 = + executeQuery( + String.format( + "source=%s | eval result = strftime(1521467703, '%s') | fields result | head 1", + TEST_INDEX_DATE, "%Y-%m-%d %H:%M:%S")); + verifyDataRows(result1, rows("2018-03-19 13:55:03")); + + // Test 2: Direct use with now() - TIMESTAMP type + JSONObject result2 = + executeQuery( + String.format( + "source=%s | eval result = strftime(now(), '%s') | fields result | head 1", + TEST_INDEX_DATE, "%Y-%m-%d")); + // Verify the result is today's date in YYYY-MM-DD format + String todayDate = LocalDate.now(ZoneOffset.UTC).toString(); + verifyDataRows(result2, rows(todayDate)); + + // Test 3: Double timestamp with milliseconds + JSONObject result3 = + executeQuery( + String.format( + "source=%s | eval result = strftime(1521467703.123456, '%s') | fields result | head" + + " 1", + TEST_INDEX_DATE, "%Y-%m-%d %H:%M:%S.%3Q")); + verifyDataRows(result3, rows("2018-03-19 13:55:03.123")); + } + + @Test + public void testStrftimeWithDateFields() throws IOException { + // Test strftime with different date field types from indices + loadIndex(Index.DATE_FORMATS); + + // Test 1: Direct use with date field (TIMESTAMP type from field) + JSONObject result1 = + executeQuery( + String.format( + "source=%s | eval formatted = strftime(epoch_millis, '%s') | " + + "fields epoch_millis, formatted | head 1", + TEST_INDEX_DATE_FORMATS, "%Y-%m-%d %H:%M:%S")); + verifyDataRows(result1, rows("1984-04-12 09:07:42.000123456", "1984-04-12 09:07:42")); + + // Test 2: Using with unix_timestamp conversion + JSONObject result2 = + executeQuery( + String.format( + "source=%s | eval unix_ts = unix_timestamp(date_time) | " + + "eval formatted = strftime(unix_ts, '%s') | " + + "fields formatted | head 1", + TEST_INDEX_DATE_FORMATS, "%F")); + verifyDataRows(result2, rows("1984-04-12")); + } + + @Test + public void testStrftimeWithExpressions() throws IOException { + // Test strftime with various expressions as input + + // Test 1: Mathematical expression + JSONObject result1 = + executeQuery( + String.format( + "source=%s | eval ts = 1521467703 + 86400 | " + + "eval result = strftime(ts, '%s') | fields result | head 1", + TEST_INDEX_DATE, "%Y-%m-%d")); + verifyDataRows(result1, rows("2018-03-20")); // One day after + + // Test 2: Conditional expression with now() + JSONObject result2 = + executeQuery( + String.format( + "source=%s | eval result = strftime(now(), '%s') | " + "fields result | head 1", + TEST_INDEX_DATE, "%Y")); + String currentYear = String.valueOf(LocalDate.now(ZoneOffset.UTC).getYear()); + verifyDataRows(result2, rows(currentYear)); + // Should return current year + + // Test 3: Using unix_timestamp to convert string first + JSONObject result3 = + executeQuery( + String.format( + "source=%s | eval ts = unix_timestamp('2018-03-19 13:55:03') | " + + "eval result = strftime(ts, '%s') | fields result | head 1", + TEST_INDEX_DATE, "%m/%d/%Y")); + verifyDataRows(result3, rows("03/19/2018")); + } + + @Test + public void testStrftimeStringHandling() throws IOException { + try { + executeQuery( + String.format( + "source=%s | eval result = strftime('1521467703', '%s') | fields result | head 1", + TEST_INDEX_DATE, "%Y-%m-%d")); + fail("String literals should not be accepted by strftime"); + } catch (Exception e) { + // Expected - string literals are not supported + // The error occurs because Calcite tries to convert the string to a timestamp + // which doesn't match the expected timestamp format + assertTrue( + "Error should indicate format issue or type problem", + e.getMessage().contains("unsupported format") + || e.getMessage().contains("timestamp") + || e.getMessage().contains("500")); + } + + // Test 2: The correct approach - use numeric literals directly + JSONObject result1 = + executeQuery( + String.format( + "source=%s | eval result = strftime(1521467703, '%s') | fields result | head 1", + TEST_INDEX_DATE, "%Y-%m-%d")); + verifyDataRows(result1, rows("2018-03-19")); + + // Test 3: For date strings, users must use unix_timestamp() first + // This is the recommended approach for converting date strings + JSONObject result2 = + executeQuery( + String.format( + "source=%s | eval ts = unix_timestamp('2018-03-19 13:55:03') | " + + "eval result = strftime(ts, '%s') | fields result | head 1", + TEST_INDEX_DATE, "%Y-%m-%d")); + verifyDataRows(result2, rows("2018-03-19")); + } + + @Test + public void testStrftimeWithNegativeTimestamps() throws IOException { + // Test strftime with negative timestamps (dates before 1970) + + // Test 1: -1 represents 1969-12-31 23:59:59 UTC + JSONObject result1 = + executeQuery( + String.format( + "source=%s | eval result = strftime(-1, '%s') | fields result | head 1", + TEST_INDEX_DATE, "%Y-%m-%d %H:%M:%S")); + verifyDataRows(result1, rows("1969-12-31 23:59:59")); + + // Test 2: -86400 represents 1969-12-31 00:00:00 UTC (one day before epoch) + JSONObject result2 = + executeQuery( + String.format( + "source=%s | eval result = strftime(-86400, '%s') | fields result | head 1", + TEST_INDEX_DATE, "%Y-%m-%d")); + verifyDataRows(result2, rows("1969-12-31")); + + // Test 3: -31536000 represents 1969-01-01 00:00:00 UTC (one year before epoch) + JSONObject result3 = + executeQuery( + String.format( + "source=%s | eval result = strftime(-31536000, '%s') | fields result | head 1", + TEST_INDEX_DATE, "%Y-%m-%d")); + verifyDataRows(result3, rows("1969-01-01")); + + // Test 4: Large negative timestamp for older dates + // -946771200 represents 1940-01-01 00:00:00 UTC + JSONObject result4 = + executeQuery( + String.format( + "source=%s | eval result = strftime(-946771200, '%s') | fields result | head 1", + TEST_INDEX_DATE, "%Y-%m-%d")); + verifyDataRows(result4, rows("1940-01-01")); + } } diff --git a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteExplainIT.java b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteExplainIT.java index f2a2fdc53ad..c4f02502813 100644 --- a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteExplainIT.java +++ b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteExplainIT.java @@ -652,4 +652,15 @@ private String executeWithReplace(String ppl) throws IOException { var result = executeQueryToString(ppl); return result.replace("\\r\\n", "\\n"); } + + @Test + public void testStrftimeFunctionExplain() throws IOException { + // Test explain for strftime function + String query = + "source=opensearch-sql_test_index_account | eval formatted_date = strftime(1521467703," + + " '%Y-%m-%d') | fields formatted_date | head 1"; + var result = explainQueryToString(query); + String expected = loadExpectedPlan("explain_strftime_function.json"); + assertJsonEqualsIgnoreId(expected, result); + } } diff --git a/integ-test/src/test/java/org/opensearch/sql/ppl/NewAddedCommandsIT.java b/integ-test/src/test/java/org/opensearch/sql/ppl/NewAddedCommandsIT.java index 6eb73ac7242..399839c26ff 100644 --- a/integ-test/src/test/java/org/opensearch/sql/ppl/NewAddedCommandsIT.java +++ b/integ-test/src/test/java/org/opensearch/sql/ppl/NewAddedCommandsIT.java @@ -125,28 +125,13 @@ public void testAppendcol() throws IOException { @Test public void testRegexMatch() throws IOException { // Test regex_match with pattern that matches substring - JSONObject result; try { - - String query1 = + executeQuery( String.format( - "source=%s | eval f=regex_match(name, 'ell') | fields f", TEST_INDEX_STRINGS); - result = executeQuery(query1); - result = - executeQuery( - String.format( - "search source=%s | where firstname = [ source=%s | where holdersName='Hattie'" - + " | fields holdersName | head 1]", - TEST_INDEX_BANK, TEST_INDEX_DOG)); + "source=%s | eval f=regex_match(name, 'ell') | fields f", TEST_INDEX_STRINGS)); } catch (ResponseException e) { - result = new JSONObject(TestUtils.getResponseBody(e.getResponse())); - if (isCalciteEnabled()) { - assertFalse(result.getJSONArray("datarows").isEmpty()); - } else { - JSONObject error = result.getJSONObject("error"); - assertThat( - error.getString("details"), containsString("unsupported function name: regex_match")); - } + JSONObject result = new JSONObject(TestUtils.getResponseBody(e.getResponse())); + verifyQuery(result); } } @@ -166,6 +151,21 @@ public void testAppend() throws IOException { verifyQuery(result); } + @Test + public void testStrftimeFunction() throws IOException { + JSONObject result; + try { + executeQuery( + String.format( + "search source=%s | eval formatted_time = strftime(1521467703, '%s') | fields" + + " formatted_time", + TEST_INDEX_BANK, "%Y-%m-%d")); + } catch (ResponseException e) { + result = new JSONObject(TestUtils.getResponseBody(e.getResponse())); + verifyQuery(result); + } + } + private void verifyQuery(JSONObject result) throws IOException { if (isCalciteEnabled()) { assertFalse(result.getJSONArray("datarows").isEmpty()); diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_strftime_function.json b/integ-test/src/test/resources/expectedOutput/calcite/explain_strftime_function.json new file mode 100644 index 00000000000..55af94637f0 --- /dev/null +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_strftime_function.json @@ -0,0 +1,6 @@ +{ + "calcite": { + "logical": "LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT])\n LogicalSort(fetch=[1])\n LogicalProject(formatted_date=[STRFTIME(1521467703, '%Y-%m-%d':VARCHAR)])\n CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]])\n", + "physical": "EnumerableCalc(expr#0..16=[{inputs}], expr#17=[1521467703], expr#18=['%Y-%m-%d':VARCHAR], expr#19=[STRFTIME($t17, $t18)], formatted_date=[$t19])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[LIMIT->1, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={\"from\":0,\"size\":1,\"timeout\":\"1m\"}, requestedTotalSize=1, pageSize=null, startFrom=0)])\n" + } +} \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_strftime_function.json b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_strftime_function.json new file mode 100644 index 00000000000..a48f7e114c8 --- /dev/null +++ b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_strftime_function.json @@ -0,0 +1,6 @@ +{ + "calcite": { + "logical": "LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT])\n LogicalSort(fetch=[1])\n LogicalProject(formatted_date=[STRFTIME(1521467703, '%Y-%m-%d':VARCHAR)])\n CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]])\n", + "physical": "EnumerableLimit(fetch=[10000])\n EnumerableCalc(expr#0..16=[{inputs}], expr#17=[1521467703], expr#18=['%Y-%m-%d':VARCHAR], expr#19=[STRFTIME($t17, $t18)], formatted_date=[$t19])\n EnumerableLimit(fetch=[1])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]])\n" + } +} \ No newline at end of file diff --git a/ppl/src/main/antlr/OpenSearchPPLLexer.g4 b/ppl/src/main/antlr/OpenSearchPPLLexer.g4 index 4c37be2f318..48b051e456b 100644 --- a/ppl/src/main/antlr/OpenSearchPPLLexer.g4 +++ b/ppl/src/main/antlr/OpenSearchPPLLexer.g4 @@ -386,6 +386,7 @@ UTC_TIME: 'UTC_TIME'; UTC_TIMESTAMP: 'UTC_TIMESTAMP'; WEEKDAY: 'WEEKDAY'; YEARWEEK: 'YEARWEEK'; +STRFTIME: 'STRFTIME'; // TEXT FUNCTIONS SUBSTR: 'SUBSTR'; diff --git a/ppl/src/main/antlr/OpenSearchPPLParser.g4 b/ppl/src/main/antlr/OpenSearchPPLParser.g4 index d5cb4e3452b..1b8399c3e7f 100644 --- a/ppl/src/main/antlr/OpenSearchPPLParser.g4 +++ b/ppl/src/main/antlr/OpenSearchPPLParser.g4 @@ -1084,6 +1084,7 @@ dateTimeFunctionName | WEEK_OF_YEAR | YEAR | YEARWEEK + | STRFTIME ; getFormatFunctionCall diff --git a/ppl/src/test/java/org/opensearch/sql/ppl/calcite/CalcitePPLFunctionTypeTest.java b/ppl/src/test/java/org/opensearch/sql/ppl/calcite/CalcitePPLFunctionTypeTest.java index 8da3818cc04..855cd17db68 100644 --- a/ppl/src/test/java/org/opensearch/sql/ppl/calcite/CalcitePPLFunctionTypeTest.java +++ b/ppl/src/test/java/org/opensearch/sql/ppl/calcite/CalcitePPLFunctionTypeTest.java @@ -5,9 +5,14 @@ package org.opensearch.sql.ppl.calcite; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import org.apache.calcite.rel.RelNode; import org.apache.calcite.test.CalciteAssert; import org.junit.Assert; import org.junit.Test; +import org.opensearch.sql.exception.ExpressionEvaluationException; public class CalcitePPLFunctionTypeTest extends CalcitePPLAbstractTest { @@ -176,6 +181,111 @@ public void testLog2WithWrongArgShouldThrow() { "LOG2 function expects {[INTEGER]|[DOUBLE]}, but got [STRING,STRING]"); } + @Test + public void testStrftimeWithCorrectTypes() { + // Test with integer timestamp and string format + getRelNode("source=EMP | eval formatted = strftime(1521467703, '%Y-%m-%d') | fields formatted"); + // Test with double timestamp and string format + getRelNode( + "source=EMP | eval formatted = strftime(1521467703.0, '%Y-%m-%d %H:%M:%S') | fields" + + " formatted"); + // Test with expression that returns numeric type + getRelNode( + "source=EMP | eval formatted = strftime(EMPNO * 1000000, '%F %T') | fields formatted"); + // Test with timestamp from now() + getRelNode("source=EMP | eval formatted = strftime(now(), '%Y-%m-%d') | fields formatted"); + // Test with timestamp from from_unixtime() + getRelNode( + "source=EMP | eval formatted = strftime(from_unixtime(1521467703), '%Y-%m-%d') | fields" + + " formatted"); + } + + @Test + public void testStrftimeWithWrongFirstArgType() { + // First argument should be numeric/timestamp, not boolean + String ppl = "source=EMP | eval formatted = strftime(EMPNO > 5, '%Y-%m-%d') | fields formatted"; + Throwable t = Assert.assertThrows(ExpressionEvaluationException.class, () -> getRelNode(ppl)); + verifyErrorMessageContains( + t, + "STRFTIME function expects {[INTEGER,STRING]|[DOUBLE,STRING]|[TIMESTAMP,STRING]}, but got" + + " [BOOLEAN,STRING]"); + } + + @Test + public void testStrftimeAcceptsDateInput() { + // DATE values are automatically converted to TIMESTAMP by Calcite + // This test verifies that DATE inputs work via auto-conversion + String ppl = + "source=EMP | eval formatted = strftime(date('2020-09-16'), '%Y-%m-%d') | fields formatted"; + RelNode relNode = getRelNode(ppl); + assertNotNull(relNode); + // The plan should show TIMESTAMP(DATE(...)) indicating auto-conversion + String planString = relNode.explain(); + assertTrue(planString.contains("STRFTIME") && planString.contains("TIMESTAMP")); + } + + @Test + public void testStrftimeWithDateReturningFunctions() { + // Test strftime with various functions that return DATE/TIMESTAMP types + + // Test with NOW() function + String ppl1 = + "source=EMP | eval formatted = strftime(now(), '%Y-%m-%d %H:%M:%S') | fields formatted"; + RelNode relNode1 = getRelNode(ppl1); + assertNotNull(relNode1); + + // Test with TIMESTAMP function + String ppl2 = + "source=EMP | eval formatted = strftime(timestamp('2020-09-16 10:30:45'), '%Y-%m-%d" + + " %H:%M:%S') | fields formatted"; + RelNode relNode2 = getRelNode(ppl2); + assertNotNull(relNode2); + + // Test with FROM_UNIXTIME (returns TIMESTAMP) + String ppl3 = + "source=EMP | eval formatted = strftime(from_unixtime(1521467703), '%Y-%m-%d %H:%M:%S') |" + + " fields formatted"; + RelNode relNode3 = getRelNode(ppl3); + assertNotNull(relNode3); + + // Test with chained date functions + String ppl4 = + "source=EMP | eval ts = timestamp('2020-09-16 10:30:45') | eval formatted = strftime(ts," + + " '%F %T') | fields formatted"; + RelNode relNode4 = getRelNode(ppl4); + assertNotNull(relNode4); + } + + @Test + public void testStrftimeWithWrongSecondArgType() { + // Second argument should be string, not numeric + String ppl = "source=EMP | eval formatted = strftime(1521467703, 123) | fields formatted"; + Throwable t = Assert.assertThrows(ExpressionEvaluationException.class, () -> getRelNode(ppl)); + verifyErrorMessageContains( + t, + "STRFTIME function expects {[INTEGER,STRING]|[DOUBLE,STRING]|[TIMESTAMP,STRING]}, but got" + + " [INTEGER,INTEGER]"); + } + + @Test + public void testStrftimeWithWrongNumberOfArgs() { + // strftime requires exactly 2 arguments + String ppl1 = "source=EMP | eval formatted = strftime(1521467703) | fields formatted"; + Throwable t1 = Assert.assertThrows(ExpressionEvaluationException.class, () -> getRelNode(ppl1)); + verifyErrorMessageContains( + t1, + "STRFTIME function expects {[INTEGER,STRING]|[DOUBLE,STRING]|[TIMESTAMP,STRING]}, but got" + + " [INTEGER]"); + + String ppl2 = + "source=EMP | eval formatted = strftime(1521467703, '%Y', 'extra') | fields formatted"; + Throwable t2 = Assert.assertThrows(ExpressionEvaluationException.class, () -> getRelNode(ppl2)); + verifyErrorMessageContains( + t2, + "STRFTIME function expects {[INTEGER,STRING]|[DOUBLE,STRING]|[TIMESTAMP,STRING]}, but got" + + " [INTEGER,STRING,STRING]"); + } + // mvjoin should reject non-string single values @Test public void testMvjoinRejectsNonStringValues() { diff --git a/ppl/src/test/java/org/opensearch/sql/ppl/utils/PPLQueryDataAnonymizerTest.java b/ppl/src/test/java/org/opensearch/sql/ppl/utils/PPLQueryDataAnonymizerTest.java index 6fa6df3fde1..5dfc73f5483 100644 --- a/ppl/src/test/java/org/opensearch/sql/ppl/utils/PPLQueryDataAnonymizerTest.java +++ b/ppl/src/test/java/org/opensearch/sql/ppl/utils/PPLQueryDataAnonymizerTest.java @@ -237,6 +237,13 @@ public void testEvalCommand() { assertEquals("source=t | eval r=abs(f)", anonymize("source=t | eval r=abs(f)")); } + @Test + public void testEvalCommandWithStrftime() { + assertEquals( + "source=t | eval formatted=strftime(timestamp,***)", + anonymize("source=t | eval formatted=strftime(timestamp, '%Y-%m-%d %H:%M:%S')")); + } + @Test public void testFillNullSameValue() { assertEquals(