diff --git a/core/src/main/java/org/opensearch/sql/analysis/Analyzer.java b/core/src/main/java/org/opensearch/sql/analysis/Analyzer.java index f7017565595..3d0c29768e6 100644 --- a/core/src/main/java/org/opensearch/sql/analysis/Analyzer.java +++ b/core/src/main/java/org/opensearch/sql/analysis/Analyzer.java @@ -96,7 +96,6 @@ import org.opensearch.sql.ast.tree.StreamWindow; import org.opensearch.sql.ast.tree.SubqueryAlias; import org.opensearch.sql.ast.tree.TableFunction; -import org.opensearch.sql.ast.tree.Timechart; import org.opensearch.sql.ast.tree.Trendline; import org.opensearch.sql.ast.tree.UnresolvedPlan; import org.opensearch.sql.ast.tree.Values; @@ -775,11 +774,6 @@ public LogicalPlan visitChart(Chart node, AnalysisContext context) { throw getOnlyForCalciteException("Chart"); } - @Override - public LogicalPlan visitTimechart(Timechart node, AnalysisContext context) { - throw getOnlyForCalciteException("Timechart"); - } - @Override public LogicalPlan visitWindow(Window node, AnalysisContext context) { throw getOnlyForCalciteException("Window"); diff --git a/core/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java b/core/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java index 320723fd57c..85fee21fa13 100644 --- a/core/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java +++ b/core/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java @@ -83,7 +83,6 @@ import org.opensearch.sql.ast.tree.StreamWindow; import org.opensearch.sql.ast.tree.SubqueryAlias; import org.opensearch.sql.ast.tree.TableFunction; -import org.opensearch.sql.ast.tree.Timechart; import org.opensearch.sql.ast.tree.Trendline; import org.opensearch.sql.ast.tree.Values; import org.opensearch.sql.ast.tree.Window; @@ -280,10 +279,6 @@ public T visitChart(Chart node, C context) { return visitChildren(node, context); } - public T visitTimechart(Timechart node, C context) { - return visitChildren(node, context); - } - public T visitRegex(Regex node, C context) { return visitChildren(node, context); } diff --git a/core/src/main/java/org/opensearch/sql/ast/dsl/AstDSL.java b/core/src/main/java/org/opensearch/sql/ast/dsl/AstDSL.java index 67cc893c5b0..78d715a76da 100644 --- a/core/src/main/java/org/opensearch/sql/ast/dsl/AstDSL.java +++ b/core/src/main/java/org/opensearch/sql/ast/dsl/AstDSL.java @@ -728,7 +728,7 @@ public static Bin bin(UnresolvedExpression field, Argument... arguments) { } /** Get a reference to the implicit timestamp field {@code @timestamp} */ - public static Field referImplicitTimestampField() { + public static Field implicitTimestampField() { return AstDSL.field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP); } } diff --git a/core/src/main/java/org/opensearch/sql/ast/tree/Chart.java b/core/src/main/java/org/opensearch/sql/ast/tree/Chart.java index d0f982edce6..ada20cbde74 100644 --- a/core/src/main/java/org/opensearch/sql/ast/tree/Chart.java +++ b/core/src/main/java/org/opensearch/sql/ast/tree/Chart.java @@ -5,17 +5,44 @@ package org.opensearch.sql.ast.tree; +import static org.opensearch.sql.ast.dsl.AstDSL.aggregate; +import static org.opensearch.sql.ast.dsl.AstDSL.doubleLiteral; +import static org.opensearch.sql.ast.dsl.AstDSL.eval; +import static org.opensearch.sql.ast.dsl.AstDSL.function; +import static org.opensearch.sql.ast.dsl.AstDSL.stringLiteral; +import static org.opensearch.sql.ast.expression.IntervalUnit.MILLISECOND; +import static org.opensearch.sql.ast.tree.Chart.PerFunctionRateExprBuilder.timestampadd; +import static org.opensearch.sql.ast.tree.Chart.PerFunctionRateExprBuilder.timestampdiff; +import static org.opensearch.sql.expression.function.BuiltinFunctionName.DIVIDE; +import static org.opensearch.sql.expression.function.BuiltinFunctionName.MULTIPLY; +import static org.opensearch.sql.expression.function.BuiltinFunctionName.SUM; +import static org.opensearch.sql.expression.function.BuiltinFunctionName.TIMESTAMPADD; +import static org.opensearch.sql.expression.function.BuiltinFunctionName.TIMESTAMPDIFF; + import com.google.common.collect.ImmutableList; import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; import lombok.AllArgsConstructor; import lombok.EqualsAndHashCode; import lombok.Getter; +import lombok.RequiredArgsConstructor; import lombok.ToString; import org.opensearch.sql.ast.AbstractNodeVisitor; import org.opensearch.sql.ast.dsl.AstDSL; +import org.opensearch.sql.ast.expression.AggregateFunction; +import org.opensearch.sql.ast.expression.Alias; import org.opensearch.sql.ast.expression.Argument; +import org.opensearch.sql.ast.expression.Field; +import org.opensearch.sql.ast.expression.Function; +import org.opensearch.sql.ast.expression.IntervalUnit; +import org.opensearch.sql.ast.expression.Let; import org.opensearch.sql.ast.expression.Literal; +import org.opensearch.sql.ast.expression.Span; +import org.opensearch.sql.ast.expression.SpanUnit; import org.opensearch.sql.ast.expression.UnresolvedExpression; +import org.opensearch.sql.calcite.utils.PlanUtils; /** AST node represent chart command. */ @Getter @@ -39,8 +66,8 @@ public class Chart extends UnresolvedPlan { @Override public UnresolvedPlan attach(UnresolvedPlan child) { - this.child = child; - return this; + // Transform after child attached to avoid unintentionally overriding it + return toBuilder().child(child).build().transformPerFunction(); } @Override @@ -52,4 +79,131 @@ public List getChild() { public T accept(AbstractNodeVisitor nodeVisitor, C context) { return nodeVisitor.visitChart(this, context); } + + /** + * Transform per function to eval-based post-processing on sum result by chart. Specifically, + * calculate how many seconds are in the time bucket based on the span option dynamically, then + * divide the aggregated sum value by the number of seconds to get the per-second rate. + * + *

For example, with span=5m per_second(field): per second rate = sum(field) / 300 seconds + * + * @return eval+chart if per function present, or the original chart otherwise. + */ + private UnresolvedPlan transformPerFunction() { + Optional perFuncOpt = PerFunction.from(aggregationFunction); + if (perFuncOpt.isEmpty()) { + return this; + } + + PerFunction perFunc = perFuncOpt.get(); + // For chart, the rowSplit should contain the span information + UnresolvedExpression spanExpr = rowSplit; + if (rowSplit instanceof Alias) { + spanExpr = ((Alias) rowSplit).getDelegated(); + } + if (!(spanExpr instanceof Span)) { + return this; // Cannot transform without span information + } + + Span span = (Span) spanExpr; + Field spanStartTime = AstDSL.implicitTimestampField(); + Function spanEndTime = timestampadd(span.getUnit(), span.getValue(), spanStartTime); + Function spanMillis = timestampdiff(MILLISECOND, spanStartTime, spanEndTime); + final int SECOND_IN_MILLISECOND = 1000; + return eval( + chart(AstDSL.alias(perFunc.aggName, PerFunctionRateExprBuilder.sum(perFunc.aggArg))), + let(perFunc.aggName) + .multiply(perFunc.seconds * SECOND_IN_MILLISECOND) + .dividedBy(spanMillis)); + } + + private Chart chart(UnresolvedExpression newAggregationFunction) { + return this.toBuilder().aggregationFunction(newAggregationFunction).build(); + } + + @RequiredArgsConstructor + static class PerFunction { + private static final Map UNIT_SECONDS = + Map.of( + "per_second", 1, + "per_minute", 60, + "per_hour", 3600, + "per_day", 86400); + private final String aggName; + private final UnresolvedExpression aggArg; + private final int seconds; + + static Optional from(UnresolvedExpression aggExpr) { + if (aggExpr instanceof Alias) { + return from(((Alias) aggExpr).getDelegated()); + } + ; + if (!(aggExpr instanceof AggregateFunction)) { + return Optional.empty(); + } + + AggregateFunction aggFunc = (AggregateFunction) aggExpr; + String aggFuncName = aggFunc.getFuncName().toLowerCase(Locale.ROOT); + if (!UNIT_SECONDS.containsKey(aggFuncName)) { + return Optional.empty(); + } + + String aggName = toAggName(aggFunc); + return Optional.of( + new PerFunction(aggName, aggFunc.getField(), UNIT_SECONDS.get(aggFuncName))); + } + + private static String toAggName(AggregateFunction aggFunc) { + String fieldName = + (aggFunc.getField() instanceof Field) + ? ((Field) aggFunc.getField()).getField().toString() + : aggFunc.getField().toString(); + return String.format(Locale.ROOT, "%s(%s)", aggFunc.getFuncName(), fieldName); + } + } + + private PerFunctionRateExprBuilder let(String fieldName) { + return new PerFunctionRateExprBuilder(AstDSL.field(fieldName)); + } + + /** Fluent builder for creating Let expressions with mathematical operations. */ + static class PerFunctionRateExprBuilder { + private final Field field; + private UnresolvedExpression expr; + + PerFunctionRateExprBuilder(Field field) { + this.field = field; + this.expr = field; + } + + PerFunctionRateExprBuilder multiply(Integer multiplier) { + // Promote to double literal to avoid integer division in downstream + this.expr = + function( + MULTIPLY.getName().getFunctionName(), expr, doubleLiteral(multiplier.doubleValue())); + return this; + } + + Let dividedBy(UnresolvedExpression divisor) { + return AstDSL.let(field, function(DIVIDE.getName().getFunctionName(), expr, divisor)); + } + + static UnresolvedExpression sum(UnresolvedExpression field) { + return aggregate(SUM.getName().getFunctionName(), field); + } + + static Function timestampadd( + SpanUnit unit, UnresolvedExpression value, UnresolvedExpression timestampField) { + UnresolvedExpression intervalUnit = + stringLiteral(PlanUtils.spanUnitToIntervalUnit(unit).toString()); + return function( + TIMESTAMPADD.getName().getFunctionName(), intervalUnit, value, timestampField); + } + + static Function timestampdiff( + IntervalUnit unit, UnresolvedExpression start, UnresolvedExpression end) { + return function( + TIMESTAMPDIFF.getName().getFunctionName(), stringLiteral(unit.toString()), start, end); + } + } } diff --git a/core/src/main/java/org/opensearch/sql/ast/tree/Timechart.java b/core/src/main/java/org/opensearch/sql/ast/tree/Timechart.java deleted file mode 100644 index 19972358721..00000000000 --- a/core/src/main/java/org/opensearch/sql/ast/tree/Timechart.java +++ /dev/null @@ -1,209 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.sql.ast.tree; - -import static org.opensearch.sql.ast.dsl.AstDSL.aggregate; -import static org.opensearch.sql.ast.dsl.AstDSL.doubleLiteral; -import static org.opensearch.sql.ast.dsl.AstDSL.eval; -import static org.opensearch.sql.ast.dsl.AstDSL.function; -import static org.opensearch.sql.ast.dsl.AstDSL.stringLiteral; -import static org.opensearch.sql.ast.expression.IntervalUnit.MILLISECOND; -import static org.opensearch.sql.ast.tree.Timechart.PerFunctionRateExprBuilder.sum; -import static org.opensearch.sql.ast.tree.Timechart.PerFunctionRateExprBuilder.timestampadd; -import static org.opensearch.sql.ast.tree.Timechart.PerFunctionRateExprBuilder.timestampdiff; -import static org.opensearch.sql.calcite.plan.OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP; -import static org.opensearch.sql.expression.function.BuiltinFunctionName.DIVIDE; -import static org.opensearch.sql.expression.function.BuiltinFunctionName.MULTIPLY; -import static org.opensearch.sql.expression.function.BuiltinFunctionName.SUM; -import static org.opensearch.sql.expression.function.BuiltinFunctionName.TIMESTAMPADD; -import static org.opensearch.sql.expression.function.BuiltinFunctionName.TIMESTAMPDIFF; - -import com.google.common.collect.ImmutableList; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Optional; -import lombok.AllArgsConstructor; -import lombok.EqualsAndHashCode; -import lombok.Getter; -import lombok.RequiredArgsConstructor; -import lombok.ToString; -import org.opensearch.sql.ast.AbstractNodeVisitor; -import org.opensearch.sql.ast.dsl.AstDSL; -import org.opensearch.sql.ast.expression.AggregateFunction; -import org.opensearch.sql.ast.expression.Field; -import org.opensearch.sql.ast.expression.Function; -import org.opensearch.sql.ast.expression.IntervalUnit; -import org.opensearch.sql.ast.expression.Let; -import org.opensearch.sql.ast.expression.Span; -import org.opensearch.sql.ast.expression.SpanUnit; -import org.opensearch.sql.ast.expression.UnresolvedExpression; -import org.opensearch.sql.calcite.utils.PlanUtils; - -/** AST node represent Timechart operation. */ -@Getter -@ToString -@EqualsAndHashCode(callSuper = false) -@AllArgsConstructor -@lombok.Builder(toBuilder = true) -public class Timechart extends UnresolvedPlan { - private UnresolvedPlan child; - private UnresolvedExpression binExpression; - private UnresolvedExpression aggregateFunction; - private UnresolvedExpression byField; - private Integer limit; - private Boolean useOther; - - public Timechart(UnresolvedPlan child, UnresolvedExpression aggregateFunction) { - this(child, null, aggregateFunction, null, null, true); - } - - public Timechart span(UnresolvedExpression binExpression) { - return toBuilder().binExpression(binExpression).build(); - } - - public Timechart by(UnresolvedExpression byField) { - return toBuilder().byField(byField).build(); - } - - public Timechart limit(Integer limit) { - return toBuilder().limit(limit).build(); - } - - public Timechart useOther(Boolean useOther) { - return toBuilder().useOther(useOther).build(); - } - - @Override - public UnresolvedPlan attach(UnresolvedPlan child) { - // Transform after child attached to avoid unintentionally overriding it - return toBuilder().child(child).build().transformPerFunction(); - } - - @Override - public List getChild() { - return ImmutableList.of(child); - } - - @Override - public T accept(AbstractNodeVisitor nodeVisitor, C context) { - return nodeVisitor.visitTimechart(this, context); - } - - /** - * Transform per function to eval-based post-processing on sum result by timechart. Specifically, - * calculate how many seconds are in the time bucket based on the span option dynamically, then - * divide the aggregated sum value by the number of seconds to get the per-second rate. - * - *

For example, with span=5m per_second(field): per second rate = sum(field) / 300 seconds - * - * @return eval+timechart if per function present, or the original timechart otherwise. - */ - private UnresolvedPlan transformPerFunction() { - Optional perFuncOpt = PerFunction.from(aggregateFunction); - if (perFuncOpt.isEmpty()) { - return this; - } - - PerFunction perFunc = perFuncOpt.get(); - Span span = (Span) this.binExpression; - Field spanStartTime = AstDSL.field(IMPLICIT_FIELD_TIMESTAMP); - Function spanEndTime = timestampadd(span.getUnit(), span.getValue(), spanStartTime); - Function spanMillis = timestampdiff(MILLISECOND, spanStartTime, spanEndTime); - final int SECOND_IN_MILLISECOND = 1000; - return eval( - timechart(AstDSL.alias(perFunc.aggName, sum(perFunc.aggArg))), - let(perFunc.aggName) - .multiply(perFunc.seconds * SECOND_IN_MILLISECOND) - .dividedBy(spanMillis)); - } - - private Timechart timechart(UnresolvedExpression newAggregateFunction) { - return this.toBuilder().aggregateFunction(newAggregateFunction).build(); - } - - @RequiredArgsConstructor - static class PerFunction { - private static final Map UNIT_SECONDS = - Map.of( - "per_second", 1, - "per_minute", 60, - "per_hour", 3600, - "per_day", 86400); - private final String aggName; - private final UnresolvedExpression aggArg; - private final int seconds; - - static Optional from(UnresolvedExpression aggExpr) { - if (!(aggExpr instanceof AggregateFunction)) { - return Optional.empty(); - } - - AggregateFunction aggFunc = (AggregateFunction) aggExpr; - String aggFuncName = aggFunc.getFuncName().toLowerCase(Locale.ROOT); - if (!UNIT_SECONDS.containsKey(aggFuncName)) { - return Optional.empty(); - } - - String aggName = toAggName(aggFunc); - return Optional.of( - new PerFunction(aggName, aggFunc.getField(), UNIT_SECONDS.get(aggFuncName))); - } - - private static String toAggName(AggregateFunction aggFunc) { - String fieldName = - (aggFunc.getField() instanceof Field) - ? ((Field) aggFunc.getField()).getField().toString() - : aggFunc.getField().toString(); - return String.format(Locale.ROOT, "%s(%s)", aggFunc.getFuncName(), fieldName); - } - } - - private PerFunctionRateExprBuilder let(String fieldName) { - return new PerFunctionRateExprBuilder(AstDSL.field(fieldName)); - } - - /** Fluent builder for creating Let expressions with mathematical operations. */ - static class PerFunctionRateExprBuilder { - private final Field field; - private UnresolvedExpression expr; - - PerFunctionRateExprBuilder(Field field) { - this.field = field; - this.expr = field; - } - - PerFunctionRateExprBuilder multiply(Integer multiplier) { - // Promote to double literal to avoid integer division in downstream - this.expr = - function( - MULTIPLY.getName().getFunctionName(), expr, doubleLiteral(multiplier.doubleValue())); - return this; - } - - Let dividedBy(UnresolvedExpression divisor) { - return AstDSL.let(field, function(DIVIDE.getName().getFunctionName(), expr, divisor)); - } - - static UnresolvedExpression sum(UnresolvedExpression field) { - return aggregate(SUM.getName().getFunctionName(), field); - } - - static Function timestampadd( - SpanUnit unit, UnresolvedExpression value, UnresolvedExpression timestampField) { - UnresolvedExpression intervalUnit = - stringLiteral(PlanUtils.spanUnitToIntervalUnit(unit).toString()); - return function( - TIMESTAMPADD.getName().getFunctionName(), intervalUnit, value, timestampField); - } - - static Function timestampdiff( - IntervalUnit unit, UnresolvedExpression start, UnresolvedExpression end) { - return function( - TIMESTAMPDIFF.getName().getFunctionName(), stringLiteral(unit.toString()), start, end); - } - } -} diff --git a/core/src/main/java/org/opensearch/sql/calcite/CalciteRelNodeVisitor.java b/core/src/main/java/org/opensearch/sql/calcite/CalciteRelNodeVisitor.java index 94086952610..dcd1d721e90 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/CalciteRelNodeVisitor.java +++ b/core/src/main/java/org/opensearch/sql/calcite/CalciteRelNodeVisitor.java @@ -31,7 +31,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.BitSet; -import java.util.Collections; import java.util.Comparator; import java.util.HashSet; import java.util.List; @@ -2499,17 +2498,13 @@ public RelNode visitChart(Chart node, CalcitePlanContext context) { relBuilder.field(0), relBuilder.alias(columnSplitExpr, columnSplitName), relBuilder.field(2)); - String aggFunctionName = getAggFunctionName(node.getAggregationFunction()); - BuiltinFunctionName aggFunction = - BuiltinFunctionName.of(aggFunctionName) - .orElseThrow( - () -> - new IllegalArgumentException( - StringUtils.format( - "Unrecognized aggregation function: %s", aggFunctionName))); relBuilder.aggregate( relBuilder.groupKey(relBuilder.field(0), relBuilder.field(1)), - buildAggCall(context.relBuilder, aggFunction, relBuilder.field(2)).as(aggFieldName)); + buildAggCall( + context.relBuilder, + getAggFunctionName(node.getAggregationFunction()), + relBuilder.field(2)) + .as(aggFieldName)); // The output of chart is expected to be ordered by row and column split names relBuilder.sort(relBuilder.field(0), relBuilder.field(1)); return relBuilder.peek(); @@ -2551,6 +2546,37 @@ private RelNode rankByColumnSplit( return relBuilder.build(); } + /** + * Aggregate a field based on a given built-in aggregation function name. + * + *

It is intended for secondary aggregations in timechart and chart commands. Using it + * elsewhere may lead to unintended results. It handles explicitly only MIN, MAX, AVG, COUNT, + * DISTINCT_COUNT, EARLIEST, and LATEST. It sums the results for the rest aggregation types, + * assuming them to be accumulative. + */ + private AggCall buildAggCall(RelBuilder relBuilder, String aggFunctionName, RexNode node) { + BuiltinFunctionName aggFunction = + BuiltinFunctionName.of(aggFunctionName) + .orElseThrow( + () -> + new IllegalArgumentException( + StringUtils.format( + "Unrecognized aggregation function: %s", aggFunctionName))); + return switch (aggFunction) { + case MIN, EARLIEST -> relBuilder.min(node); + case MAX, LATEST -> relBuilder.max(node); + case AVG -> relBuilder.avg(node); + default -> relBuilder.sum(node); + }; + } + + private String getAggFunctionName(UnresolvedExpression aggregateFunction) { + if (aggregateFunction instanceof Alias alias) { + return getAggFunctionName(alias.getDelegated()); + } + return ((AggregateFunction) aggregateFunction).getFuncName(); + } + @AllArgsConstructor private static class ChartConfig { private final int limit; @@ -2573,355 +2599,6 @@ static ChartConfig fromArguments(ArgumentMap argMap) { } } - /** Transforms timechart command into SQL-based operations. */ - @Override - public RelNode visitTimechart( - org.opensearch.sql.ast.tree.Timechart node, CalcitePlanContext context) { - visitChildren(node, context); - - // Extract parameters - UnresolvedExpression spanExpr = node.getBinExpression(); - - List groupExprList; - - // Handle no by field case - if (node.getByField() == null) { - String aggFieldAlias = getAggFieldAlias(node.getAggregateFunction()); - - // Create group expression list with just the timestamp span but use a different alias - // to avoid @timestamp naming conflict - List simpleGroupExprList = new ArrayList<>(); - simpleGroupExprList.add(new Alias("timestamp", spanExpr)); - // Create agg expression list with the aggregate function - List simpleAggExprList = - List.of(new Alias(aggFieldAlias, node.getAggregateFunction())); - // Create an Aggregation object - Aggregation aggregation = - new Aggregation( - simpleAggExprList, - Collections.emptyList(), - simpleGroupExprList, - null, - Collections.emptyList()); - // Use visitAggregation to handle the aggregation and column naming - RelNode result = visitAggregation(aggregation, context); - // Push the result and add explicit projection to get [@timestamp, count] order - context.relBuilder.push(result); - // Reorder fields: timestamp first, then count - context.relBuilder.project( - context.relBuilder.field("timestamp"), context.relBuilder.field(aggFieldAlias)); - // Rename timestamp to @timestamp - context.relBuilder.rename(List.of("@timestamp", aggFieldAlias)); - - context.relBuilder.sort(context.relBuilder.field(0)); - return context.relBuilder.peek(); - } - - // Extract parameters for byField case - UnresolvedExpression byField = node.getByField(); - String byFieldName = ((Field) byField).getField().toString(); - String aggFieldAlias = getAggFieldAlias(node.getAggregateFunction()); - - int limit = Optional.ofNullable(node.getLimit()).orElse(10); - boolean useOther = Optional.ofNullable(node.getUseOther()).orElse(true); - - try { - // Step 1: Initial aggregation - IMPORTANT: order is [spanExpr, byField] - groupExprList = Arrays.asList(spanExpr, byField); - aggregateWithTrimming(groupExprList, List.of(node.getAggregateFunction()), context, false); - - // First rename the timestamp field (2nd to last) to @timestamp - List fieldNames = context.relBuilder.peek().getRowType().getFieldNames(); - List renamedFields = new ArrayList<>(fieldNames); - // TODO: Fix aggregateWithTrimming reordering - renamedFields.set(fieldNames.size() - 2, "@timestamp"); - context.relBuilder.rename(renamedFields); - - // Then reorder: @timestamp first, then byField, then value function - List outputFields = context.relBuilder.fields(); - List reordered = new ArrayList<>(); - reordered.add(context.relBuilder.field("@timestamp")); // timestamp first - reordered.add(context.relBuilder.field(byFieldName)); // byField second - reordered.add(outputFields.get(outputFields.size() - 1)); // value function last - context.relBuilder.project(reordered); - - // Handle no limit case - just sort and return with proper field aliases - if (limit == 0) { - // Add final projection with proper aliases: [@timestamp, byField, aggFieldAlias] - context.relBuilder.project( - context.relBuilder.alias(context.relBuilder.field(0), "@timestamp"), - context.relBuilder.alias(context.relBuilder.field(1), byFieldName), - context.relBuilder.alias(context.relBuilder.field(2), aggFieldAlias)); - context.relBuilder.sort(context.relBuilder.field(0), context.relBuilder.field(1)); - return context.relBuilder.peek(); - } - - // Use known field positions after reordering: 0=@timestamp, 1=byField, 2=value - RelNode completeResults = context.relBuilder.build(); - - // Step 2: Find top N categories using window function approach (more efficient than separate - // aggregation) - String aggFunctionName = getAggFunctionName(node.getAggregateFunction()); - Optional aggFuncNameOptional = BuiltinFunctionName.of(aggFunctionName); - if (aggFuncNameOptional.isEmpty()) { - throw new IllegalArgumentException( - StringUtils.format("Unrecognized aggregation function: %s", aggFunctionName)); - } - BuiltinFunctionName aggFunction = aggFuncNameOptional.get(); - RelNode topCategories = buildTopCategoriesQuery(completeResults, limit, aggFunction, context); - - // Step 3: Apply OTHER logic with single pass - return buildFinalResultWithOther( - completeResults, - topCategories, - byFieldName, - aggFunction, - aggFieldAlias, - useOther, - limit, - context); - - } catch (Exception e) { - throw new RuntimeException("Error in visitTimechart: " + e.getMessage(), e); - } - } - - private String getAggFunctionName(UnresolvedExpression aggregateFunction) { - if (aggregateFunction instanceof Alias alias) { - return getAggFunctionName(alias.getDelegated()); - } - return ((AggregateFunction) aggregateFunction).getFuncName(); - } - - /** Build top categories query - simpler approach that works better with OTHER handling */ - private RelNode buildTopCategoriesQuery( - RelNode completeResults, - int limit, - BuiltinFunctionName aggFunction, - CalcitePlanContext context) { - context.relBuilder.push(completeResults); - - // Filter out null values when determining top categories - null should not count towards limit - context.relBuilder.filter(context.relBuilder.isNotNull(context.relBuilder.field(1))); - - // Get totals for non-null categories - field positions: 0=@timestamp, 1=byField, 2=value - RexInputRef valueField = context.relBuilder.field(2); - AggCall call = buildAggCall(context.relBuilder, aggFunction, valueField); - - context.relBuilder.aggregate( - context.relBuilder.groupKey(context.relBuilder.field(1)), call.as("grand_total")); - - // Apply sorting and limit to non-null categories only - RexNode sortField = context.relBuilder.field("grand_total"); - // For MIN and EARLIEST, top results should be the minimum ones - sortField = - aggFunction == BuiltinFunctionName.MIN || aggFunction == BuiltinFunctionName.EARLIEST - ? sortField - : context.relBuilder.desc(sortField); - context.relBuilder.sort(sortField); - if (limit > 0) { - context.relBuilder.limit(0, limit); - } - return context.relBuilder.build(); - } - - /** Build final result with OTHER category using efficient single-pass approach */ - private RelNode buildFinalResultWithOther( - RelNode completeResults, - RelNode topCategories, - String byFieldName, - BuiltinFunctionName aggFunction, - String aggFieldAlias, - boolean useOther, - int limit, - CalcitePlanContext context) { - - // Use zero-filling for count aggregations, standard result for others - if (aggFieldAlias.equals("count")) { - return buildZeroFilledResult( - completeResults, topCategories, byFieldName, aggFieldAlias, useOther, limit, context); - } else { - return buildStandardResult( - completeResults, - topCategories, - byFieldName, - aggFunction, - aggFieldAlias, - useOther, - context); - } - } - - /** Build standard result without zero-filling */ - private RelNode buildStandardResult( - RelNode completeResults, - RelNode topCategories, - String byFieldName, - BuiltinFunctionName aggFunctionName, - String aggFieldAlias, - boolean useOther, - CalcitePlanContext context) { - - context.relBuilder.push(completeResults); - context.relBuilder.push(topCategories); - - // LEFT JOIN to identify top categories - field positions: 0=@timestamp, 1=byField, 2=value - context.relBuilder.join( - org.apache.calcite.rel.core.JoinRelType.LEFT, - context.relBuilder.equals( - context.relBuilder.field(2, 0, 1), context.relBuilder.field(2, 1, 0))); - - // Calculate field position after join - int topCategoryFieldIndex = completeResults.getRowType().getFieldCount(); - - // Create CASE expression for OTHER logic - RexNode categoryExpr = createOtherCaseExpression(topCategoryFieldIndex, 1, context); - - // Project and aggregate - context.relBuilder.project( - context.relBuilder.alias(context.relBuilder.field(0), "@timestamp"), - context.relBuilder.alias(categoryExpr, byFieldName), - context.relBuilder.alias(context.relBuilder.field(2), aggFieldAlias)); - - RexInputRef valueField = context.relBuilder.field(2); - AggCall aggCall = buildAggCall(context.relBuilder, aggFunctionName, valueField); - context.relBuilder.aggregate( - context.relBuilder.groupKey(context.relBuilder.field(0), context.relBuilder.field(1)), - aggCall.as(aggFieldAlias)); - - applyFiltersAndSort(useOther, context); - return context.relBuilder.peek(); - } - - /** Helper to create OTHER case expression - preserves NULL as a category */ - private RexNode createOtherCaseExpression( - int topCategoryFieldIndex, int byIndex, CalcitePlanContext context) { - return context.relBuilder.call( - org.apache.calcite.sql.fun.SqlStdOperatorTable.CASE, - context.relBuilder.isNotNull(context.relBuilder.field(topCategoryFieldIndex)), - context.relBuilder.field(byIndex), // Keep original value (including NULL) - context.relBuilder.call( - org.apache.calcite.sql.fun.SqlStdOperatorTable.CASE, - context.relBuilder.isNull(context.relBuilder.field(byIndex)), - context.relBuilder.literal(null), // Preserve NULL as NULL - context.relBuilder.literal("OTHER"))); - } - - /** Helper to apply filters and sorting */ - private void applyFiltersAndSort(boolean useOther, CalcitePlanContext context) { - if (!useOther) { - context.relBuilder.filter( - context.relBuilder.notEquals( - context.relBuilder.field(1), context.relBuilder.literal("OTHER"))); - } - context.relBuilder.sort(context.relBuilder.field(0), context.relBuilder.field(1)); - } - - /** Build zero-filled result using fillnull pattern - treat NULL as just another category */ - private RelNode buildZeroFilledResult( - RelNode completeResults, - RelNode topCategories, - String byFieldName, - String aggFieldAlias, - boolean useOther, - int limit, - CalcitePlanContext context) { - - // Get all unique timestamps - field positions: 0=@timestamp, 1=byField, 2=value - context.relBuilder.push(completeResults); - context.relBuilder.aggregate(context.relBuilder.groupKey(context.relBuilder.field(0))); - RelNode allTimestamps = context.relBuilder.build(); - - // Get all categories for zero-filling - apply OTHER logic here too - context.relBuilder.push(completeResults); - context.relBuilder.push(topCategories); - context.relBuilder.join( - org.apache.calcite.rel.core.JoinRelType.LEFT, - context.relBuilder.call( - org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_DISTINCT_FROM, - context.relBuilder.field(2, 0, 1), - context.relBuilder.field(2, 1, 0))); - - int topCategoryFieldIndex = completeResults.getRowType().getFieldCount(); - RexNode categoryExpr = createOtherCaseExpression(topCategoryFieldIndex, 1, context); - - context.relBuilder.project(categoryExpr); - context.relBuilder.aggregate(context.relBuilder.groupKey(context.relBuilder.field(0))); - RelNode allCategories = context.relBuilder.build(); - - // Cross join timestamps with ALL categories (including OTHER) for zero-filling - context.relBuilder.push(allTimestamps); - context.relBuilder.push(allCategories); - context.relBuilder.join( - org.apache.calcite.rel.core.JoinRelType.INNER, context.relBuilder.literal(true)); - - // Create zero-filled combinations with count=0 - context.relBuilder.project( - context.relBuilder.alias( - context.relBuilder.cast(context.relBuilder.field(0), SqlTypeName.TIMESTAMP), - "@timestamp"), - context.relBuilder.alias(context.relBuilder.field(1), byFieldName), - context.relBuilder.alias(context.relBuilder.literal(0), aggFieldAlias)); - RelNode zeroFilledCombinations = context.relBuilder.build(); - - // Get actual results with OTHER logic applied - context.relBuilder.push(completeResults); - context.relBuilder.push(topCategories); - context.relBuilder.join( - org.apache.calcite.rel.core.JoinRelType.LEFT, - // Use IS NOT DISTINCT FROM for proper null handling in join - context.relBuilder.call( - org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_DISTINCT_FROM, - context.relBuilder.field(2, 0, 1), - context.relBuilder.field(2, 1, 0))); - - int actualTopCategoryFieldIndex = completeResults.getRowType().getFieldCount(); - RexNode actualCategoryExpr = createOtherCaseExpression(actualTopCategoryFieldIndex, 1, context); - - context.relBuilder.project( - context.relBuilder.alias( - context.relBuilder.cast(context.relBuilder.field(0), SqlTypeName.TIMESTAMP), - "@timestamp"), - context.relBuilder.alias(actualCategoryExpr, byFieldName), - context.relBuilder.alias(context.relBuilder.field(2), aggFieldAlias)); - - context.relBuilder.aggregate( - context.relBuilder.groupKey(context.relBuilder.field(0), context.relBuilder.field(1)), - context.relBuilder.sum(context.relBuilder.field(2)).as("actual_count")); - RelNode actualResults = context.relBuilder.build(); - - // UNION zero-filled with actual results - context.relBuilder.push(actualResults); - context.relBuilder.push(zeroFilledCombinations); - context.relBuilder.union(false); - - // Aggregate to combine actual and zero-filled data - context.relBuilder.aggregate( - context.relBuilder.groupKey(context.relBuilder.field(0), context.relBuilder.field(1)), - context.relBuilder.sum(context.relBuilder.field(2)).as(aggFieldAlias)); - - applyFiltersAndSort(useOther, context); - return context.relBuilder.peek(); - } - - /** - * Aggregate a field based on a given built-in aggregation function name. - * - *

It is intended for secondary aggregations in timechart and chart commands. Using it - * elsewhere may lead to unintended results. It handles explicitly only MIN, MAX, AVG, COUNT, - * DISTINCT_COUNT, EARLIEST, and LATEST. It sums the results for the rest aggregation types, - * assuming them to be accumulative. - */ - private AggCall buildAggCall( - RelBuilder relBuilder, BuiltinFunctionName aggFunction, RexNode node) { - return switch (aggFunction) { - case MIN, EARLIEST -> relBuilder.min(node); - case MAX, LATEST -> relBuilder.max(node); - case AVG -> relBuilder.avg(node); - default -> relBuilder.sum(node); - }; - } - @Override public RelNode visitTrendline(Trendline node, CalcitePlanContext context) { visitChildren(node, context); diff --git a/core/src/test/java/org/opensearch/sql/ast/tree/TimechartTest.java b/core/src/test/java/org/opensearch/sql/ast/tree/PerFunctionsTest.java similarity index 70% rename from core/src/test/java/org/opensearch/sql/ast/tree/TimechartTest.java rename to core/src/test/java/org/opensearch/sql/ast/tree/PerFunctionsTest.java index d587ff71787..2fa3bc2cd24 100644 --- a/core/src/test/java/org/opensearch/sql/ast/tree/TimechartTest.java +++ b/core/src/test/java/org/opensearch/sql/ast/tree/PerFunctionsTest.java @@ -13,7 +13,9 @@ import static org.opensearch.sql.ast.dsl.AstDSL.function; import static org.opensearch.sql.ast.dsl.AstDSL.intLiteral; import static org.opensearch.sql.ast.dsl.AstDSL.relation; +import static org.opensearch.sql.calcite.plan.OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP; +import java.util.List; import java.util.stream.Stream; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -21,13 +23,15 @@ import org.junit.jupiter.params.provider.MethodSource; import org.opensearch.sql.ast.dsl.AstDSL; import org.opensearch.sql.ast.expression.AggregateFunction; +import org.opensearch.sql.ast.expression.Argument; import org.opensearch.sql.ast.expression.Let; +import org.opensearch.sql.ast.expression.Literal; import org.opensearch.sql.ast.expression.Span; import org.opensearch.sql.ast.expression.SpanUnit; import org.opensearch.sql.ast.expression.UnresolvedExpression; +import org.opensearch.sql.expression.function.BuiltinFunctionName; -class TimechartTest { - +class PerFunctionsTest { /** * @return test sources for per_* function test. */ @@ -57,8 +61,9 @@ void should_transform_per_second_for_different_spans( multiply("per_second(bytes)", 1000.0), timestampdiff( "MILLISECOND", - "@timestamp", - timestampadd(expectedIntervalUnit, spanValue, "@timestamp")))), + IMPLICIT_FIELD_TIMESTAMP, + timestampadd( + expectedIntervalUnit, spanValue, IMPLICIT_FIELD_TIMESTAMP)))), timechart(span(spanValue, spanUnit), alias("per_second(bytes)", sum("bytes"))))); } @@ -76,8 +81,9 @@ void should_transform_per_minute_for_different_spans( multiply("per_minute(bytes)", 60000.0), timestampdiff( "MILLISECOND", - "@timestamp", - timestampadd(expectedIntervalUnit, spanValue, "@timestamp")))), + IMPLICIT_FIELD_TIMESTAMP, + timestampadd( + expectedIntervalUnit, spanValue, IMPLICIT_FIELD_TIMESTAMP)))), timechart(span(spanValue, spanUnit), alias("per_minute(bytes)", sum("bytes"))))); } @@ -95,8 +101,9 @@ void should_transform_per_hour_for_different_spans( multiply("per_hour(bytes)", 3600000.0), timestampdiff( "MILLISECOND", - "@timestamp", - timestampadd(expectedIntervalUnit, spanValue, "@timestamp")))), + IMPLICIT_FIELD_TIMESTAMP, + timestampadd( + expectedIntervalUnit, spanValue, IMPLICIT_FIELD_TIMESTAMP)))), timechart(span(spanValue, spanUnit), alias("per_hour(bytes)", sum("bytes"))))); } @@ -114,8 +121,9 @@ void should_transform_per_day_for_different_spans( multiply("per_day(bytes)", 8.64E7), timestampdiff( "MILLISECOND", - "@timestamp", - timestampadd(expectedIntervalUnit, spanValue, "@timestamp")))), + IMPLICIT_FIELD_TIMESTAMP, + timestampadd( + expectedIntervalUnit, spanValue, IMPLICIT_FIELD_TIMESTAMP)))), timechart(span(spanValue, spanUnit), alias("per_day(bytes)", sum("bytes"))))); } @@ -128,19 +136,27 @@ void should_not_transform_non_per_functions() { @Test void should_preserve_all_fields_during_per_function_transformation() { - Timechart original = - new Timechart(relation("logs"), perSecond("bytes")) - .span(span(5, "m")) - .by(field("status")) - .limit(20) - .useOther(false); - - Timechart expected = - new Timechart(relation("logs"), alias("per_second(bytes)", sum("bytes"))) - .span(span(5, "m")) - .by(field("status")) - .limit(20) - .useOther(false); + Chart original = + Chart.builder() + .child(relation("logs")) + .aggregationFunction(perSecond("bytes")) + .rowSplit(span(5, "m")) + .columnSplit(field("status")) + .arguments( + List.of( + new Argument("limit", intLiteral(20)), new Argument("useOther", Literal.FALSE))) + .build(); + + Chart expected = + Chart.builder() + .child(relation("logs")) + .aggregationFunction(alias("per_second(bytes)", sum("bytes"))) + .rowSplit(span(5, "m")) + .columnSplit(field("status")) + .arguments( + List.of( + new Argument("limit", intLiteral(20)), new Argument("useOther", Literal.FALSE))) + .build(); withTimechart(original) .whenTransformingPerFunction() @@ -151,7 +167,9 @@ void should_preserve_all_fields_during_per_function_transformation() { divide( multiply("per_second(bytes)", 1000.0), timestampdiff( - "MILLISECOND", "@timestamp", timestampadd("MINUTE", 5, "@timestamp")))), + "MILLISECOND", + IMPLICIT_FIELD_TIMESTAMP, + timestampadd("MINUTE", 5, IMPLICIT_FIELD_TIMESTAMP)))), expected)); } @@ -161,17 +179,21 @@ private static TransformationAssertion withTimechart(Span spanExpr, AggregateFun return new TransformationAssertion(timechart(spanExpr, aggFunc)); } - private static TransformationAssertion withTimechart(Timechart timechart) { + private static TransformationAssertion withTimechart(Chart timechart) { return new TransformationAssertion(timechart); } - private static Timechart timechart(Span spanExpr, UnresolvedExpression aggExpr) { + private static Chart timechart(Span spanExpr, UnresolvedExpression aggExpr) { // Set child here because expected object won't call attach below - return new Timechart(relation("t"), aggExpr).span(spanExpr).limit(10).useOther(true); + return Chart.builder() + .child(relation("t")) + .aggregationFunction(aggExpr) + .rowSplit(spanExpr) + .build(); } private static Span span(int value, String unit) { - return AstDSL.span(field("@timestamp"), intLiteral(value), SpanUnit.of(unit)); + return AstDSL.span(AstDSL.implicitTimestampField(), intLiteral(value), SpanUnit.of(unit)); } private static AggregateFunction perSecond(String fieldName) { @@ -209,23 +231,31 @@ private static UnresolvedExpression divide( private static UnresolvedExpression timestampadd(String unit, int value, String timestampField) { return function( - "timestampadd", AstDSL.stringLiteral(unit), intLiteral(value), field(timestampField)); + BuiltinFunctionName.TIMESTAMPADD.getName().getFunctionName(), + AstDSL.stringLiteral(unit), + intLiteral(value), + field(timestampField)); } private static UnresolvedExpression timestampdiff( String unit, String startField, UnresolvedExpression end) { - return function("timestampdiff", AstDSL.stringLiteral(unit), field(startField), end); + + return function( + BuiltinFunctionName.TIMESTAMPDIFF.getName().getFunctionName(), + AstDSL.stringLiteral(unit), + field(startField), + end); } - private static UnresolvedPlan eval(Let letExpr, Timechart timechartExpr) { + private static UnresolvedPlan eval(Let letExpr, Chart timechartExpr) { return AstDSL.eval(timechartExpr, letExpr); } private static class TransformationAssertion { - private final Timechart timechart; + private final Chart timechart; private UnresolvedPlan result; - TransformationAssertion(Timechart timechart) { + TransformationAssertion(Chart timechart) { this.timechart = timechart; } diff --git a/docs/user/dql/metadata.rst b/docs/user/dql/metadata.rst index 3b277cd978f..7584c72505e 100644 --- a/docs/user/dql/metadata.rst +++ b/docs/user/dql/metadata.rst @@ -35,31 +35,33 @@ Example 1: Show All Indices Information SQL query:: os> SHOW TABLES LIKE '%' - fetched rows / total rows = 20/20 - +----------------+-------------+------------------+------------+---------+----------+------------+-----------+---------------------------+----------------+ - | TABLE_CAT | TABLE_SCHEM | TABLE_NAME | TABLE_TYPE | REMARKS | TYPE_CAT | TYPE_SCHEM | TYPE_NAME | SELF_REFERENCING_COL_NAME | REF_GENERATION | - |----------------+-------------+------------------+------------+---------+----------+------------+-----------+---------------------------+----------------| - | docTestCluster | null | .ql-datasources | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | account2 | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | accounts | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | apache | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | books | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | events | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | json_test | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | nested | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | nyc_taxi | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | occupation | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | otellogs | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | people | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | state_country | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | time_data | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | time_data2 | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | time_test | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | weblogs | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | wildcard | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | work_information | BASE TABLE | null | null | null | null | null | null | - | docTestCluster | null | worker | BASE TABLE | null | null | null | null | null | null | - +----------------+-------------+------------------+------------+---------+----------+------------+-----------+---------------------------+----------------+ + fetched rows / total rows = 22/22 + +----------------+-------------+-------------------+------------+---------+----------+------------+-----------+---------------------------+----------------+ + | TABLE_CAT | TABLE_SCHEM | TABLE_NAME | TABLE_TYPE | REMARKS | TYPE_CAT | TYPE_SCHEM | TYPE_NAME | SELF_REFERENCING_COL_NAME | REF_GENERATION | + |----------------+-------------+-------------------+------------+---------+----------+------------+-----------+---------------------------+----------------| + | docTestCluster | null | .ql-datasources | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | account2 | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | accounts | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | apache | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | books | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | events | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | events_many_hosts | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | events_null | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | json_test | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | nested | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | nyc_taxi | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | occupation | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | otellogs | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | people | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | state_country | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | time_data | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | time_data2 | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | time_test | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | weblogs | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | wildcard | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | work_information | BASE TABLE | null | null | null | null | null | null | + | docTestCluster | null | worker | BASE TABLE | null | null | null | null | null | null | + +----------------+-------------+-------------------+------------+---------+----------+------------+-----------+---------------------------+----------------+ Example 2: Show Specific Index Information ------------------------------------------ diff --git a/docs/user/ppl/cmd/timechart.rst b/docs/user/ppl/cmd/timechart.rst index 512fa76370c..6aea1270f1e 100644 --- a/docs/user/ppl/cmd/timechart.rst +++ b/docs/user/ppl/cmd/timechart.rst @@ -132,109 +132,94 @@ PPL query:: os> source=events | timechart span=1h count() by host fetched rows / total rows = 2/2 - +---------------------+---------+-------+ - | @timestamp | host | count | - |---------------------+---------+-------| - | 2023-01-01 10:00:00 | server1 | 4 | - | 2023-01-01 10:00:00 | server2 | 4 | - +---------------------+---------+-------+ - -Example 2: Count events by minute with zero-filled results + +---------------------+---------+---------+ + | @timestamp | host | count() | + |---------------------+---------+---------| + | 2023-01-01 10:00:00 | server1 | 4 | + | 2023-01-01 10:00:00 | server2 | 4 | + +---------------------+---------+---------+ + +Example 2: Count events by minute ========================================================== -This example counts events for each minute and groups them by host, showing zero values for time-host combinations with no data. +This example counts events for each minute and groups them by host. PPL query:: os> source=events | timechart span=1m count() by host - fetched rows / total rows = 16/16 - +---------------------+---------+-------+ - | @timestamp | host | count | - |---------------------+---------+-------| - | 2023-01-01 10:00:00 | server1 | 1 | - | 2023-01-01 10:00:00 | server2 | 0 | - | 2023-01-01 10:05:00 | server1 | 0 | - | 2023-01-01 10:05:00 | server2 | 1 | - | 2023-01-01 10:10:00 | server1 | 1 | - | 2023-01-01 10:10:00 | server2 | 0 | - | 2023-01-01 10:15:00 | server1 | 0 | - | 2023-01-01 10:15:00 | server2 | 1 | - | 2023-01-01 10:20:00 | server1 | 1 | - | 2023-01-01 10:20:00 | server2 | 0 | - | 2023-01-01 10:25:00 | server1 | 0 | - | 2023-01-01 10:25:00 | server2 | 1 | - | 2023-01-01 10:30:00 | server1 | 1 | - | 2023-01-01 10:30:00 | server2 | 0 | - | 2023-01-01 10:35:00 | server1 | 0 | - | 2023-01-01 10:35:00 | server2 | 1 | - +---------------------+---------+-------+ - -Example 3: Calculate average CPU usage by minute + fetched rows / total rows = 8/8 + +---------------------+---------+---------+ + | @timestamp | host | count() | + |---------------------+---------+---------| + | 2023-01-01 10:00:00 | server1 | 1 | + | 2023-01-01 10:05:00 | server2 | 1 | + | 2023-01-01 10:10:00 | server1 | 1 | + | 2023-01-01 10:15:00 | server2 | 1 | + | 2023-01-01 10:20:00 | server1 | 1 | + | 2023-01-01 10:25:00 | server2 | 1 | + | 2023-01-01 10:30:00 | server1 | 1 | + | 2023-01-01 10:35:00 | server2 | 1 | + +---------------------+---------+---------+ + +Example 3: Calculate average number of packets by minute ================================================ -This example calculates the average CPU usage for each minute without grouping by any field. +This example calculates the average packets for each minute without grouping by any field. PPL query:: - PPL> source=events | timechart span=1m avg(cpu_usage) - fetched rows / total rows = 5/5 - +---------------------+------------------+ - | @timestamp | avg(cpu_usage) | - +---------------------+------------------+ - | 2024-07-01 00:00:00 | 45.2 | - | 2024-07-01 00:01:00 | 38.7 | - | 2024-07-01 00:02:00 | 55.3 | - | 2024-07-01 00:03:00 | 42.1 | - | 2024-07-01 00:04:00 | 41.8 | - +---------------------+------------------+ - -Example 4: Calculate average CPU usage by second and region + os> source=events | timechart span=1m avg(packets) + fetched rows / total rows = 8/8 + +---------------------+--------------+ + | @timestamp | avg(packets) | + |---------------------+--------------| + | 2023-01-01 10:00:00 | 60.0 | + | 2023-01-01 10:05:00 | 30.0 | + | 2023-01-01 10:10:00 | 60.0 | + | 2023-01-01 10:15:00 | 30.0 | + | 2023-01-01 10:20:00 | 60.0 | + | 2023-01-01 10:25:00 | 30.0 | + | 2023-01-01 10:30:00 | 180.0 | + | 2023-01-01 10:35:00 | 90.0 | + +---------------------+--------------+ + +Example 4: Calculate average number of packets by every 20 minutes and status =========================================================== -This example calculates the average CPU usage for each second and groups them by region. +This example calculates the average number of packets for every 20 minutes and groups them by status. PPL query:: - PPL> source=events | timechart span=1s avg(cpu_usage) by region - fetched rows / total rows = 5/5 - +---------------------+---------+------------------+ - | @timestamp | region | avg(cpu_usage) | - +---------------------+---------+------------------+ - | 2024-07-01 00:00:00 | us-east | 45.2 | - | 2024-07-01 00:01:00 | us-west | 38.7 | - | 2024-07-01 00:02:00 | us-east | 55.3 | - | 2024-07-01 00:03:00 | eu-west | 42.1 | - | 2024-07-01 00:04:00 | us-west | 41.8 | - +---------------------+---------+------------------+ - -Example 5: Count events by second and region with zero-filled results + os> source=events | timechart span=20m avg(packets) by status + fetched rows / total rows = 8/8 + +---------------------+------------+--------------+ + | @timestamp | status | avg(packets) | + |---------------------+------------+--------------| + | 2023-01-01 10:00:00 | active | 30.0 | + | 2023-01-01 10:00:00 | inactive | 30.0 | + | 2023-01-01 10:00:00 | pending | 60.0 | + | 2023-01-01 10:00:00 | processing | 60.0 | + | 2023-01-01 10:20:00 | cancelled | 180.0 | + | 2023-01-01 10:20:00 | completed | 60.0 | + | 2023-01-01 10:20:00 | inactive | 90.0 | + | 2023-01-01 10:20:00 | pending | 30.0 | + +---------------------+------------+--------------+ + +Example 5: Count events by hour and category ===================================================================== -This example counts events for each second and groups them by region, showing zero values for time-region combinations with no data. +This example counts events for each second and groups them by category PPL query:: - PPL> source=events | timechart span=1s count() by region - fetched rows / total rows = 15/15 - +---------------------+---------+-------+ - | @timestamp | region | count | - +---------------------+---------+-------+ - | 2024-07-01 00:00:00 | us-east | 1 | - | 2024-07-01 00:00:00 | us-west | 0 | - | 2024-07-01 00:00:00 | eu-west | 0 | - | 2024-07-01 00:01:00 | us-east | 0 | - | 2024-07-01 00:01:00 | us-west | 1 | - | 2024-07-01 00:01:00 | eu-west | 0 | - | 2024-07-01 00:02:00 | us-east | 1 | - | 2024-07-01 00:02:00 | us-west | 0 | - | 2024-07-01 00:02:00 | eu-west | 0 | - | 2024-07-01 00:03:00 | us-east | 0 | - | 2024-07-01 00:03:00 | us-west | 0 | - | 2024-07-01 00:03:00 | eu-west | 1 | - | 2024-07-01 00:04:00 | us-east | 0 | - | 2024-07-01 00:04:00 | us-west | 1 | - | 2024-07-01 00:04:00 | eu-west | 0 | - +---------------------+---------+-------+ + os> source=events | timechart span=1h count() by category + fetched rows / total rows = 2/2 + +---------------------+----------+---------+ + | @timestamp | category | count() | + |---------------------+----------+---------| + | 2023-01-01 10:00:00 | orders | 4 | + | 2023-01-01 10:00:00 | users | 4 | + +---------------------+----------+---------+ Example 6: Using the limit parameter with count() function ========================================================== @@ -245,27 +230,19 @@ This query will display the top 2 hosts with the highest count values, and group PPL query:: os> source=events | timechart span=1m limit=2 count() by host - fetched rows / total rows = 16/16 - +---------------------+---------+-------+ - | @timestamp | host | count | - |---------------------+---------+-------| - | 2023-01-01 10:00:00 | server1 | 1 | - | 2023-01-01 10:00:00 | server2 | 0 | - | 2023-01-01 10:05:00 | server1 | 0 | - | 2023-01-01 10:05:00 | server2 | 1 | - | 2023-01-01 10:10:00 | server1 | 1 | - | 2023-01-01 10:10:00 | server2 | 0 | - | 2023-01-01 10:15:00 | server1 | 0 | - | 2023-01-01 10:15:00 | server2 | 1 | - | 2023-01-01 10:20:00 | server1 | 1 | - | 2023-01-01 10:20:00 | server2 | 0 | - | 2023-01-01 10:25:00 | server1 | 0 | - | 2023-01-01 10:25:00 | server2 | 1 | - | 2023-01-01 10:30:00 | server1 | 1 | - | 2023-01-01 10:30:00 | server2 | 0 | - | 2023-01-01 10:35:00 | server1 | 0 | - | 2023-01-01 10:35:00 | server2 | 1 | - +---------------------+---------+-------+ + fetched rows / total rows = 8/8 + +---------------------+---------+---------+ + | @timestamp | host | count() | + |---------------------+---------+---------| + | 2023-01-01 10:00:00 | server1 | 1 | + | 2023-01-01 10:05:00 | server2 | 1 | + | 2023-01-01 10:10:00 | server1 | 1 | + | 2023-01-01 10:15:00 | server2 | 1 | + | 2023-01-01 10:20:00 | server1 | 1 | + | 2023-01-01 10:25:00 | server2 | 1 | + | 2023-01-01 10:30:00 | server1 | 1 | + | 2023-01-01 10:35:00 | server2 | 1 | + +---------------------+---------+---------+ Example 7: Using limit=0 with count() to show all values ======================================================== @@ -274,23 +251,23 @@ To display all distinct values without any limit, set limit=0: PPL query:: - PPL> source=events_many_hosts | timechart span=1h limit=0 count() by host + os> source=events_many_hosts | timechart span=1h limit=0 count() by host fetched rows / total rows = 11/11 - +---------------------+--------+-------+ - | @timestamp | host | count | - +---------------------+--------+-------+ - | 2024-07-01 00:00:00 | web-01 | 1 | - | 2024-07-01 00:00:00 | web-02 | 1 | - | 2024-07-01 00:00:00 | web-03 | 1 | - | 2024-07-01 00:00:00 | web-04 | 1 | - | 2024-07-01 00:00:00 | web-05 | 1 | - | 2024-07-01 00:00:00 | web-06 | 1 | - | 2024-07-01 00:00:00 | web-07 | 1 | - | 2024-07-01 00:00:00 | web-08 | 1 | - | 2024-07-01 00:00:00 | web-09 | 1 | - | 2024-07-01 00:00:00 | web-10 | 1 | - | 2024-07-01 00:00:00 | web-11 | 1 | - +---------------------+--------+-------+ + +---------------------+--------+---------+ + | @timestamp | host | count() | + |---------------------+--------+---------| + | 2024-07-01 00:00:00 | web-01 | 1 | + | 2024-07-01 00:00:00 | web-02 | 1 | + | 2024-07-01 00:00:00 | web-03 | 1 | + | 2024-07-01 00:00:00 | web-04 | 1 | + | 2024-07-01 00:00:00 | web-05 | 1 | + | 2024-07-01 00:00:00 | web-06 | 1 | + | 2024-07-01 00:00:00 | web-07 | 1 | + | 2024-07-01 00:00:00 | web-08 | 1 | + | 2024-07-01 00:00:00 | web-09 | 1 | + | 2024-07-01 00:00:00 | web-10 | 1 | + | 2024-07-01 00:00:00 | web-11 | 1 | + +---------------------+--------+---------+ This shows all 11 hosts as separate rows without an "OTHER" category. @@ -301,22 +278,22 @@ Limit to top 10 hosts without OTHER category (useother=false): PPL query:: - PPL> source=events_many_hosts | timechart span=1h useother=false count() by host + os> source=events_many_hosts | timechart span=1h useother=false count() by host fetched rows / total rows = 10/10 - +---------------------+--------+-------+ - | @timestamp | host | count | - +---------------------+--------+-------+ - | 2024-07-01 00:00:00 | web-01 | 1 | - | 2024-07-01 00:00:00 | web-02 | 1 | - | 2024-07-01 00:00:00 | web-03 | 1 | - | 2024-07-01 00:00:00 | web-04 | 1 | - | 2024-07-01 00:00:00 | web-05 | 1 | - | 2024-07-01 00:00:00 | web-06 | 1 | - | 2024-07-01 00:00:00 | web-07 | 1 | - | 2024-07-01 00:00:00 | web-08 | 1 | - | 2024-07-01 00:00:00 | web-09 | 1 | - | 2024-07-01 00:00:00 | web-10 | 1 | - +---------------------+--------+-------+ + +---------------------+--------+---------+ + | @timestamp | host | count() | + |---------------------+--------+---------| + | 2024-07-01 00:00:00 | web-01 | 1 | + | 2024-07-01 00:00:00 | web-02 | 1 | + | 2024-07-01 00:00:00 | web-03 | 1 | + | 2024-07-01 00:00:00 | web-04 | 1 | + | 2024-07-01 00:00:00 | web-05 | 1 | + | 2024-07-01 00:00:00 | web-06 | 1 | + | 2024-07-01 00:00:00 | web-07 | 1 | + | 2024-07-01 00:00:00 | web-08 | 1 | + | 2024-07-01 00:00:00 | web-09 | 1 | + | 2024-07-01 00:00:00 | web-10 | 1 | + +---------------------+--------+---------+ Example 9: Using limit with useother parameter and avg() function ================================================================= @@ -325,48 +302,49 @@ Limit to top 3 hosts with OTHER category (default useother=true): PPL query:: - PPL> source=events_many_hosts | timechart span=1h limit=3 avg(cpu_usage) by host + os> source=events_many_hosts | timechart span=1h limit=3 avg(cpu_usage) by host fetched rows / total rows = 4/4 - +---------------------+--------+------------------+ - | @timestamp | host | avg(cpu_usage) | - +---------------------+--------+------------------+ - | 2024-07-01 00:00:00 | web-03 | 55.3 | - | 2024-07-01 00:00:00 | web-07 | 48.6 | - | 2024-07-01 00:00:00 | web-09 | 67.8 | - | 2024-07-01 00:00:00 | OTHER | 330.4 | - +---------------------+--------+------------------+ + +---------------------+--------+----------------+ + | @timestamp | host | avg(cpu_usage) | + |---------------------+--------+----------------| + | 2024-07-01 00:00:00 | OTHER | 41.3 | + | 2024-07-01 00:00:00 | web-03 | 55.3 | + | 2024-07-01 00:00:00 | web-07 | 48.6 | + | 2024-07-01 00:00:00 | web-09 | 67.8 | + +---------------------+--------+----------------+ Limit to top 3 hosts without OTHER category (useother=false): PPL query:: - PPL> source=events_many_hosts | timechart span=1h limit=3 useother=false avg(cpu_usage) by host + os> source=events_many_hosts | timechart span=1h limit=3 useother=false avg(cpu_usage) by host fetched rows / total rows = 3/3 - +---------------------+--------+------------------+ - | @timestamp | host | avg(cpu_usage) | - +---------------------+--------+------------------+ - | 2024-07-01 00:00:00 | web-03 | 55.3 | - | 2024-07-01 00:00:00 | web-07 | 48.6 | - | 2024-07-01 00:00:00 | web-09 | 67.8 | - +---------------------+--------+------------------+ + +---------------------+--------+----------------+ + | @timestamp | host | avg(cpu_usage) | + |---------------------+--------+----------------| + | 2024-07-01 00:00:00 | web-03 | 55.3 | + | 2024-07-01 00:00:00 | web-07 | 48.6 | + | 2024-07-01 00:00:00 | web-09 | 67.8 | + +---------------------+--------+----------------+ Example 10: Handling null values in the "by" field ================================================== This example shows how null values in the "by" field are treated as a separate category. The dataset events_null has 1 entry that does not have a host field. +It is put into a separate "NULL" category because the defaults for ``usenull`` and ``nullstr`` are ``true`` and ``"NULL"`` respectively. PPL query:: - PPL> source=events_null | timechart span=1h count() by host + os> source=events_null | timechart span=1h count() by host fetched rows / total rows = 4/4 - +---------------------+--------+-------+ - | @timestamp | host | count | - +---------------------+--------+-------+ - | 2024-07-01 00:00:00 | db-01 | 1 | - | 2024-07-01 00:00:00 | web-01 | 2 | - | 2024-07-01 00:00:00 | web-02 | 2 | - | 2024-07-01 00:00:00 | null | 1 | - +---------------------+--------+-------+ + +---------------------+--------+---------+ + | @timestamp | host | count() | + |---------------------+--------+---------| + | 2024-07-01 00:00:00 | NULL | 1 | + | 2024-07-01 00:00:00 | db-01 | 1 | + | 2024-07-01 00:00:00 | web-01 | 2 | + | 2024-07-01 00:00:00 | web-02 | 2 | + +---------------------+--------+---------+ Example 11: Calculate packets per second rate ============================================= diff --git a/doctest/test_data/events_many_hosts.json b/doctest/test_data/events_many_hosts.json new file mode 100644 index 00000000000..9c1bd8974a6 --- /dev/null +++ b/doctest/test_data/events_many_hosts.json @@ -0,0 +1,22 @@ +{"index":{"_id":"1"}} +{"@timestamp":"2024-07-01T00:00:00","host":"web-01","cpu_usage":45.2} +{"index":{"_id":"2"}} +{"@timestamp":"2024-07-01T00:00:00","host":"web-02","cpu_usage":38.7} +{"index":{"_id":"3"}} +{"@timestamp":"2024-07-01T00:00:00","host":"web-03","cpu_usage":55.3} +{"index":{"_id":"4"}} +{"@timestamp":"2024-07-01T00:00:00","host":"web-04","cpu_usage":42.1} +{"index":{"_id":"5"}} +{"@timestamp":"2024-07-01T00:00:00","host":"web-05","cpu_usage":41.8} +{"index":{"_id":"6"}} +{"@timestamp":"2024-07-01T00:00:00","host":"web-06","cpu_usage":39.4} +{"index":{"_id":"7"}} +{"@timestamp":"2024-07-01T00:00:00","host":"web-07","cpu_usage":48.6} +{"index":{"_id":"8"}} +{"@timestamp":"2024-07-01T00:00:00","host":"web-08","cpu_usage":44.2} +{"index":{"_id":"9"}} +{"@timestamp":"2024-07-01T00:00:00","host":"web-09","cpu_usage":67.8} +{"index":{"_id":"10"}} +{"@timestamp":"2024-07-01T00:00:00","host":"web-10","cpu_usage":35.9} +{"index":{"_id":"11"}} +{"@timestamp":"2024-07-01T00:00:00","host":"web-11","cpu_usage":43.1} diff --git a/doctest/test_data/events_null.json b/doctest/test_data/events_null.json new file mode 100644 index 00000000000..d5e030621db --- /dev/null +++ b/doctest/test_data/events_null.json @@ -0,0 +1,12 @@ +{"index":{"_id":"1"}} +{"@timestamp":"2024-07-01T00:00:00","host":"web-01","cpu_usage":45.2,"region":"us-east"} +{"index":{"_id":"2"}} +{"@timestamp":"2024-07-01T00:01:00","host":"web-02","cpu_usage":38.7,"region":"us-west"} +{"index":{"_id":"3"}} +{"@timestamp":"2024-07-01T00:02:00","host":"web-01","cpu_usage":55.3,"region":"us-east"} +{"index":{"_id":"4"}} +{"@timestamp":"2024-07-01T00:03:00","host":"db-01","cpu_usage":42.1,"region":"eu-west"} +{"index":{"_id":"5"}} +{"@timestamp":"2024-07-01T00:04:00","host":"web-02","cpu_usage":41.8,"region":"us-west"} +{"index":{"_id":"6"}} +{"@timestamp":"2024-07-01T00:05:00","cpu_usage":50.0,"region":"us-east"} diff --git a/doctest/test_docs.py b/doctest/test_docs.py index c2d1112b584..4fd9c230ff6 100644 --- a/doctest/test_docs.py +++ b/doctest/test_docs.py @@ -41,6 +41,8 @@ 'worker': 'worker.json', 'work_information': 'work_information.json', 'events': 'events.json', + 'events_null': 'events_null.json', + 'events_many_hosts': 'events_many_hosts.json', 'otellogs': 'otellogs.json', 'time_data': 'time_test_data.json', 'time_data2': 'time_test_data2.json', diff --git a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteChartCommandIT.java b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteChartCommandIT.java index b325912e7d7..e687751ef0c 100644 --- a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteChartCommandIT.java +++ b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteChartCommandIT.java @@ -169,8 +169,8 @@ public void testChartLimit0WithUseOther() throws IOException { JSONObject result = executeQuery( String.format( - "source=%s | chart limit=0 useother=true otherstr='max_among_other'" - + " max(severityNumber) over flags by severityText", + "source=%s | chart max(severityNumber) over flags by severityText limit=0" + + " useother=true otherstr='max_among_other'", TEST_INDEX_OTEL_LOGS)); verifySchema( result, @@ -250,7 +250,8 @@ public void testChartLimitTopWithMinAgg() throws IOException { JSONObject result = executeQuery( String.format( - "source=%s | chart limit=top2 min(severityNumber) over flags by severityText", + "source=%s | chart limit=top2 min(severityNumber) over flags by severityText" + + " useother=true otherstr='OTHER'", TEST_INDEX_OTEL_LOGS)); verifySchema( result, @@ -306,8 +307,8 @@ public void testChartUseNullFalseWithNullStr() throws IOException { JSONObject result = executeQuery( String.format( - "source=%s | chart usenull=false nullstr='not_shown' count() over gender by age" - + " span=10", + "source=%s | chart usenull=false count() over gender by age " + + " span=10 nullstr='not_shown'", TEST_INDEX_BANK_WITH_NULL_VALUES)); verifySchema( result, schema("gender", "string"), schema("age", "string"), schema("count()", "bigint")); diff --git a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteTimechartCommandIT.java b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteTimechartCommandIT.java index 4d9352e9e87..3b5c5f55475 100644 --- a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteTimechartCommandIT.java +++ b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteTimechartCommandIT.java @@ -5,9 +5,8 @@ package org.opensearch.sql.calcite.remote; -import static org.junit.jupiter.api.Assertions.*; import static org.opensearch.sql.legacy.TestUtils.*; -import static org.opensearch.sql.legacy.TestsConstants.*; +import static org.opensearch.sql.legacy.TestsConstants.TEST_INDEX_BANK; import static org.opensearch.sql.util.MatcherUtils.*; import java.io.IOException; @@ -25,9 +24,10 @@ public void init() throws Exception { disallowCalciteFallback(); // Create events index with timestamp data - createEventsIndex(); + loadIndex(Index.BANK); + loadIndex(Index.EVENTS); + loadIndex(Index.EVENTS_NULL); createEventsManyHostsIndex(); - createEventsNullIndex(); } @Test @@ -37,13 +37,12 @@ public void testTimechartWithHourSpanAndGroupBy() throws IOException { result, schema("@timestamp", "timestamp"), schema("host", "string"), - schema("count", "bigint")); + schema("count()", "bigint")); verifyDataRows( result, rows("2024-07-01 00:00:00", "db-01", 1), rows("2024-07-01 00:00:00", "web-01", 2), rows("2024-07-01 00:00:00", "web-02", 2)); - assertEquals(3, result.getInt("total")); } @Test @@ -53,51 +52,26 @@ public void testTimechartWithMinuteSpanAndGroupBy() throws IOException { result, schema("@timestamp", "timestamp"), schema("host", "string"), - schema("count", "bigint")); - - // For count aggregation with default limit (no OTHER needed): 3 hosts × 5 time spans = 15 rows - assertEquals(15, result.getInt("total")); + schema("count()", "bigint")); verifyDataRows( result, - rows("2024-07-01 00:00:00", "db-01", 0), rows("2024-07-01 00:00:00", "web-01", 1), - rows("2024-07-01 00:00:00", "web-02", 0), - rows("2024-07-01 00:01:00", "db-01", 0), - rows("2024-07-01 00:01:00", "web-01", 0), rows("2024-07-01 00:01:00", "web-02", 1), - rows("2024-07-01 00:02:00", "db-01", 0), rows("2024-07-01 00:02:00", "web-01", 1), - rows("2024-07-01 00:02:00", "web-02", 0), rows("2024-07-01 00:03:00", "db-01", 1), - rows("2024-07-01 00:03:00", "web-01", 0), - rows("2024-07-01 00:03:00", "web-02", 0), - rows("2024-07-01 00:04:00", "db-01", 0), - rows("2024-07-01 00:04:00", "web-01", 0), rows("2024-07-01 00:04:00", "web-02", 1)); } @Test public void testTimechartWithoutTimestampField() throws IOException { - // Create index without @timestamp field - String noTimestampMapping = - "{\"mappings\":{\"properties\":{\"name\":{\"type\":\"keyword\"},\"occupation\":{\"type\":\"keyword\"},\"country\":{\"type\":\"keyword\"},\"salary\":{\"type\":\"integer\"},\"year\":{\"type\":\"integer\"},\"month\":{\"type\":\"integer\"}}}}"; - if (!isIndexExist(client(), "no_timestamp")) { - createIndexByRestClient(client(), "no_timestamp", noTimestampMapping); - loadDataByRestClient(client(), "no_timestamp", "src/test/resources/occupation.json"); - } - - // Test should throw exception for missing @timestamp field Throwable exception = - assertThrowsWithReplace( + assertThrows( ResponseException.class, () -> { - executeQuery("source=no_timestamp | timechart count()"); + executeQuery(String.format("source=%s | timechart count()", TEST_INDEX_BANK)); }); - assertTrue( - "Error message should mention missing @timestamp field", - exception.getMessage().contains("@timestamp") - || exception.getMessage().contains("timestamp")); + verifyErrorMessageContains(exception, "Field [@timestamp] not found."); } @Test @@ -111,7 +85,6 @@ public void testTimechartWithMinuteSpanNoGroupBy() throws IOException { rows("2024-07-01 00:02:00", 55.3), rows("2024-07-01 00:03:00", 42.1), rows("2024-07-01 00:04:00", 41.8)); - assertEquals(5, result.getInt("total")); } @Test @@ -121,28 +94,15 @@ public void testTimechartWithSpanCountGroupBy() throws IOException { result, schema("@timestamp", "timestamp"), schema("region", "string"), - schema("count", "bigint")); - // For count aggregation with 3 regions (< default limit 10), should show zero-filled results: 3 - // regions × 5 time spans = 15 rows - assertEquals(15, result.getInt("total")); + schema("count()", "bigint")); verifyDataRows( result, rows("2024-07-01 00:00:00", "us-east", 1), - rows("2024-07-01 00:00:00", "us-west", 0), - rows("2024-07-01 00:00:00", "eu-west", 0), - rows("2024-07-01 00:01:00", "us-east", 0), rows("2024-07-01 00:01:00", "us-west", 1), - rows("2024-07-01 00:01:00", "eu-west", 0), rows("2024-07-01 00:02:00", "us-east", 1), - rows("2024-07-01 00:02:00", "us-west", 0), - rows("2024-07-01 00:02:00", "eu-west", 0), - rows("2024-07-01 00:03:00", "us-east", 0), - rows("2024-07-01 00:03:00", "us-west", 0), rows("2024-07-01 00:03:00", "eu-west", 1), - rows("2024-07-01 00:04:00", "us-east", 0), - rows("2024-07-01 00:04:00", "us-west", 1), - rows("2024-07-01 00:04:00", "eu-west", 0)); + rows("2024-07-01 00:04:00", "us-west", 1)); } @Test @@ -157,19 +117,8 @@ public void testTimechartWithOtherCategory() throws IOException { schema("avg(cpu_usage)", "double")); // Verify we have 11 data rows (10 hosts + OTHER) - assertEquals(11, result.getJSONArray("datarows").length()); - - // Verify the OTHER row exists with the correct value - boolean foundOther = false; - for (int i = 0; i < result.getJSONArray("datarows").length(); i++) { - Object[] row = result.getJSONArray("datarows").getJSONArray(i).toList().toArray(); - if ("OTHER".equals(row[1])) { - foundOther = true; - assertEquals(35.9, ((Number) row[2]).doubleValue(), 0.01); - break; - } - } - assertTrue("OTHER category not found in results", foundOther); + verifyNumOfRows(result, 11); + verifyDataRowsSome(result, rows("2024-07-01 00:00:00", "OTHER", 35.9)); } @Test @@ -183,13 +132,13 @@ public void testTimechartWithLimit() throws IOException { schema("host", "string"), schema("avg(cpu_usage)", "double")); - verifyDataRows( + verifyDataRowsInOrder( result, rows("2024-07-01 00:00:00", "web-01", 45.2), - rows("2024-07-01 00:01:00", "OTHER", 38.7), + rows("2024-07-01 00:01:00", "web-02", 38.7), rows("2024-07-01 00:02:00", "web-01", 55.3), - rows("2024-07-01 00:03:00", "db-01", 42.1), - rows("2024-07-01 00:04:00", "OTHER", 41.8)); + rows("2024-07-01 00:03:00", "OTHER", 42.1), + rows("2024-07-01 00:04:00", "web-02", 41.8)); } @Test @@ -200,29 +149,15 @@ public void testTimechartWithLimitCountGroupBy() throws IOException { result, schema("@timestamp", "timestamp"), schema("host", "string"), - schema("count", "bigint")); - - // For count with limit=2, should show zero-filled results: 3 hosts (web-01, web-02, OTHER) × 5 - // time spans = 15 rows - assertEquals(15, result.getInt("total")); + schema("count()", "bigint")); verifyDataRows( result, rows("2024-07-01 00:00:00", "web-01", 1), - rows("2024-07-01 00:00:00", "web-02", 0), - rows("2024-07-01 00:00:00", "OTHER", 0), - rows("2024-07-01 00:01:00", "web-01", 0), rows("2024-07-01 00:01:00", "web-02", 1), - rows("2024-07-01 00:01:00", "OTHER", 0), rows("2024-07-01 00:02:00", "web-01", 1), - rows("2024-07-01 00:02:00", "web-02", 0), - rows("2024-07-01 00:02:00", "OTHER", 0), - rows("2024-07-01 00:03:00", "web-01", 0), - rows("2024-07-01 00:03:00", "web-02", 0), rows("2024-07-01 00:03:00", "OTHER", 1), - rows("2024-07-01 00:04:00", "web-01", 0), - rows("2024-07-01 00:04:00", "web-02", 1), - rows("2024-07-01 00:04:00", "OTHER", 0)); + rows("2024-07-01 00:04:00", "web-02", 1)); } @Test @@ -239,7 +174,7 @@ public void testTimechartWithLimitZeroAndAvg() throws IOException { schema("avg(cpu_usage)", "double")); // Verify we have 11 data rows (all 11 hosts, no OTHER) - assertEquals(11, result.getJSONArray("datarows").length()); + verifyNumOfRows(result, 11); // Verify no OTHER category boolean foundOther = false; @@ -263,10 +198,9 @@ public void testTimechartWithLimitZeroAndCount() throws IOException { result, schema("@timestamp", "timestamp"), schema("host", "string"), - schema("count", "bigint")); + schema("count()", "bigint")); - // For count with limit=0, should show zero-filled results: 11 hosts × 1 time span = 11 rows - assertEquals(11, result.getInt("total")); + verifyNumOfRows(result, 11); } @Test @@ -282,7 +216,7 @@ public void testTimechartWithUseOtherFalseAndAvg() throws IOException { schema("avg(cpu_usage)", "double")); // Verify we have 10 data rows (top 10 hosts, no OTHER) - assertEquals(10, result.getJSONArray("datarows").length()); + verifyNumOfRows(result, 10); // Verify no OTHER category boolean foundOther = false; @@ -305,11 +239,9 @@ public void testTimechartWithUseOtherFalseAndCount() throws IOException { result, schema("@timestamp", "timestamp"), schema("host", "string"), - schema("count", "bigint")); + schema("count()", "bigint")); - // For count with useother=false, should show zero-filled results: 10 hosts × 1 time span = 10 - // rows - assertEquals(10, result.getInt("total")); + verifyNumOfRows(result, 10); } @Test @@ -320,28 +252,15 @@ public void testTimechartWithCountNoLimitByHostShowZero() throws IOException { result, schema("@timestamp", "timestamp"), schema("host", "string"), - schema("count", "bigint")); - - // For count aggregation, should show zero-filled results: 3 hosts × 5 time spans = 15 rows - assertEquals(15, result.getInt("total")); + schema("count()", "bigint")); verifyDataRows( result, rows("2024-07-01 00:00:00", "web-01", 1), - rows("2024-07-01 00:00:00", "web-02", 0), - rows("2024-07-01 00:00:00", "db-01", 0), - rows("2024-07-01 00:01:00", "web-01", 0), rows("2024-07-01 00:01:00", "web-02", 1), - rows("2024-07-01 00:01:00", "db-01", 0), rows("2024-07-01 00:02:00", "web-01", 1), - rows("2024-07-01 00:02:00", "web-02", 0), - rows("2024-07-01 00:02:00", "db-01", 0), - rows("2024-07-01 00:03:00", "web-01", 0), - rows("2024-07-01 00:03:00", "web-02", 0), rows("2024-07-01 00:03:00", "db-01", 1), - rows("2024-07-01 00:04:00", "web-01", 0), - rows("2024-07-01 00:04:00", "web-02", 1), - rows("2024-07-01 00:04:00", "db-01", 0)); + rows("2024-07-01 00:04:00", "web-02", 1)); } @Test @@ -357,106 +276,68 @@ public void testTimechartWithLimitAndUseOther() throws IOException { schema("host", "string"), schema("avg(cpu_usage)", "double")); - // Verify we have 4 data rows (3 hosts + OTHER) - assertEquals(4, result.getJSONArray("datarows").length()); - - // Verify specific values with tolerance for floating point precision - boolean foundOther = false, foundWeb03 = false, foundWeb07 = false, foundWeb09 = false; - for (int i = 0; i < result.getJSONArray("datarows").length(); i++) { - Object[] row = result.getJSONArray("datarows").getJSONArray(i).toList().toArray(); - String host = (String) row[1]; - double cpuUsage = ((Number) row[2]).doubleValue(); - - if ("OTHER".equals(host)) { - foundOther = true; - assertEquals(41.3, cpuUsage, 0.1); - } else if ("web-03".equals(host)) { - foundWeb03 = true; - assertEquals(55.3, cpuUsage, 0.1); - } else if ("web-07".equals(host)) { - foundWeb07 = true; - assertEquals(48.6, cpuUsage, 0.1); - } else if ("web-09".equals(host)) { - foundWeb09 = true; - assertEquals(67.8, cpuUsage, 0.1); - } - } - assertTrue("OTHER not found", foundOther); - assertTrue("web-03 not found", foundWeb03); - assertTrue("web-07 not found", foundWeb07); - assertTrue("web-09 not found", foundWeb09); + verifyDataRows( + result, + closeTo("2024-07-01 00:00:00", "OTHER", 41.300000000000004), + closeTo("2024-07-01 00:00:00", "web-03", 55.3), + closeTo("2024-07-01 00:00:00", "web-07", 48.6), + closeTo("2024-07-01 00:00:00", "web-09", 67.8)); } @Test public void testTimechartWithMissingHostValues() throws IOException { - createEventsNullIndex(); - JSONObject result = executeQuery("source=events_null | timechart span=1d count() by host"); verifySchema( result, schema("@timestamp", "timestamp"), schema("host", "string"), - schema("count", "bigint")); + schema("count()", "bigint")); verifyDataRows( result, rows("2024-07-01 00:00:00", "db-01", 1), rows("2024-07-01 00:00:00", "web-01", 2), rows("2024-07-01 00:00:00", "web-02", 2), - rows("2024-07-01 00:00:00", null, 1)); - - assertEquals(4, result.getInt("total")); + rows("2024-07-01 00:00:00", "NULL", 1)); } @Test public void testTimechartWithNullAndOther() throws IOException { - createEventsNullIndex(); - JSONObject result = - executeQuery("source=events_null | timechart span=1d limit=2 count() by host"); + executeQuery("source=events_null | timechart count() by host span=1d limit=2"); verifySchema( result, schema("@timestamp", "timestamp"), schema("host", "string"), - schema("count", "bigint")); + schema("count()", "bigint")); verifyDataRows( result, rows("2024-07-01 00:00:00", "OTHER", 1), rows("2024-07-01 00:00:00", "web-01", 2), rows("2024-07-01 00:00:00", "web-02", 2), - rows("2024-07-01 00:00:00", null, 1)); - - assertEquals(4, result.getInt("total")); + rows("2024-07-01 00:00:00", "NULL", 1)); } @Test public void testTimechartWithNullAndLimit() throws IOException { - createEventsNullIndex(); - JSONObject result = - executeQuery("source=events_null | timechart span=1d limit=3 count() by host"); + executeQuery("source=events_null | timechart span=1d count() by host limit=3"); verifySchema( result, schema("@timestamp", "timestamp"), schema("host", "string"), - schema("count", "bigint")); + schema("count()", "bigint")); verifyDataRows( result, rows("2024-07-01 00:00:00", "db-01", 1), rows("2024-07-01 00:00:00", "web-01", 2), rows("2024-07-01 00:00:00", "web-02", 2), - rows("2024-07-01 00:00:00", null, 1)); - - assertEquals(4, result.getInt("total")); - } - - private void createEventsIndex() throws IOException { - loadIndex(Index.EVENTS); + rows("2024-07-01 00:00:00", "NULL", 1)); } private void createEventsManyHostsIndex() throws IOException { @@ -468,13 +349,4 @@ private void createEventsManyHostsIndex() throws IOException { client(), "events_many_hosts", "src/test/resources/events_many_hosts.json"); } } - - private void createEventsNullIndex() throws IOException { - String eventsMapping = - "{\"mappings\":{\"properties\":{\"@timestamp\":{\"type\":\"date\"},\"host\":{\"type\":\"text\"},\"cpu_usage\":{\"type\":\"double\"},\"region\":{\"type\":\"keyword\"}}}}"; - if (!isIndexExist(client(), "events_null")) { - createIndexByRestClient(client(), "events_null", eventsMapping); - loadDataByRestClient(client(), "events_null", "src/test/resources/events_null.json"); - } - } } diff --git a/integ-test/src/test/java/org/opensearch/sql/util/MatcherUtils.java b/integ-test/src/test/java/org/opensearch/sql/util/MatcherUtils.java index 4e7d72ae530..fb5c0be026b 100644 --- a/integ-test/src/test/java/org/opensearch/sql/util/MatcherUtils.java +++ b/integ-test/src/test/java/org/opensearch/sql/util/MatcherUtils.java @@ -184,6 +184,12 @@ public static void verifyDataRowsInOrder(JSONObject response, Matcher verifyInOrder(response.getJSONArray("datarows"), matchers); } + @SafeVarargs + @SuppressWarnings("unchecked") + public static void verifyDataRowsSome(JSONObject response, Matcher... matchers) { + verifySome(response.getJSONArray("datarows"), matchers); + } + public static void verifyNumOfRows(JSONObject response, int numOfRow) { assertEquals(numOfRow, response.getJSONArray("datarows").length()); } diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart.yaml index a315860aac9..bd9187b4908 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart.yaml @@ -3,37 +3,38 @@ calcite: LogicalSystemLimit(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000], type=[QUERY_SIZE_LIMIT]) LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC]) LogicalAggregate(group=[{0, 1}], avg(cpu_usage)=[AVG($2)]) - LogicalProject(@timestamp=[$0], host=[CASE(IS NOT NULL($3), $1, CASE(IS NULL($1), null:NULL, 'OTHER'))], avg(cpu_usage)=[$2]) + LogicalProject(@timestamp=[$0], host=[CASE(IS NULL($1), 'NULL', <=($5, 10), $1, 'OTHER')], avg(cpu_usage)=[$2]) LogicalJoin(condition=[=($1, $3)], joinType=[left]) - LogicalProject(@timestamp=[$1], host=[$0], $f2=[$2]) - LogicalAggregate(group=[{0, 2}], agg#0=[AVG($1)]) - LogicalProject(host=[$4], cpu_usage=[$7], $f3=[SPAN($1, 1, 'm')]) - CalciteLogicalIndexScan(table=[[OpenSearch, events]]) - LogicalSort(sort0=[$1], dir0=[DESC], fetch=[10]) - LogicalAggregate(group=[{1}], grand_total=[AVG($2)]) - LogicalFilter(condition=[IS NOT NULL($1)]) - LogicalProject(@timestamp=[$1], host=[$0], $f2=[$2]) - LogicalAggregate(group=[{0, 2}], agg#0=[AVG($1)]) - LogicalProject(host=[$4], cpu_usage=[$7], $f3=[SPAN($1, 1, 'm')]) - CalciteLogicalIndexScan(table=[[OpenSearch, events]]) + LogicalProject(@timestamp=[$1], host=[$0], avg(cpu_usage)=[$2]) + LogicalAggregate(group=[{0, 2}], avg(cpu_usage)=[AVG($1)]) + LogicalProject(host=[$4], cpu_usage=[$7], @timestamp0=[SPAN($1, 1, 'm')]) + LogicalFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($7))]) + CalciteLogicalIndexScan(table=[[OpenSearch, events]]) + LogicalProject(host=[$0], __grand_total__=[$1], _row_number_chart_=[ROW_NUMBER() OVER (ORDER BY $1 DESC NULLS LAST)]) + LogicalAggregate(group=[{0}], __grand_total__=[SUM($1)]) + LogicalFilter(condition=[IS NOT NULL($0)]) + LogicalProject(host=[$0], avg(cpu_usage)=[$2]) + LogicalAggregate(group=[{0, 2}], avg(cpu_usage)=[AVG($1)]) + LogicalProject(host=[$4], cpu_usage=[$7], @timestamp0=[SPAN($1, 1, 'm')]) + LogicalFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($7))]) + CalciteLogicalIndexScan(table=[[OpenSearch, events]]) physical: | EnumerableLimit(fetch=[10000]) EnumerableSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC]) EnumerableCalc(expr#0..3=[{inputs}], expr#4=[0], expr#5=[=($t3, $t4)], expr#6=[null:DOUBLE], expr#7=[CASE($t5, $t6, $t2)], expr#8=[/($t7, $t3)], proj#0..1=[{exprs}], avg(cpu_usage)=[$t8]) EnumerableAggregate(group=[{0, 1}], agg#0=[$SUM0($2)], agg#1=[COUNT($2)]) - EnumerableCalc(expr#0..4=[{inputs}], expr#5=[IS NOT NULL($t3)], expr#6=[IS NULL($t1)], expr#7=[null:NULL], expr#8=['OTHER'], expr#9=[CASE($t6, $t7, $t8)], expr#10=[CASE($t5, $t1, $t9)], @timestamp=[$t0], host=[$t10], avg(cpu_usage)=[$t2]) + EnumerableCalc(expr#0..4=[{inputs}], expr#5=[IS NULL($t1)], expr#6=['NULL'], expr#7=[10], expr#8=[<=($t4, $t7)], expr#9=['OTHER'], expr#10=[CASE($t5, $t6, $t8, $t1, $t9)], @timestamp=[$t0], host=[$t10], avg(cpu_usage)=[$t2]) EnumerableMergeJoin(condition=[=($1, $3)], joinType=[left]) EnumerableSort(sort0=[$1], dir0=[ASC]) - EnumerableCalc(expr#0..3=[{inputs}], expr#4=[0], expr#5=[=($t3, $t4)], expr#6=[null:DOUBLE], expr#7=[CASE($t5, $t6, $t2)], expr#8=[/($t7, $t3)], @timestamp=[$t1], host=[$t0], $f2=[$t8]) + EnumerableCalc(expr#0..3=[{inputs}], expr#4=[0], expr#5=[=($t3, $t4)], expr#6=[null:DOUBLE], expr#7=[CASE($t5, $t6, $t2)], expr#8=[/($t7, $t3)], @timestamp=[$t1], host=[$t0], avg(cpu_usage)=[$t8]) EnumerableAggregate(group=[{0, 2}], agg#0=[$SUM0($1)], agg#1=[COUNT($1)]) - EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], expr#4=['m'], expr#5=[SPAN($t2, $t3, $t4)], proj#0..1=[{exprs}], $f2=[$t5]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[host, cpu_usage, @timestamp]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["host","cpu_usage","@timestamp"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], expr#4=['m'], expr#5=[SPAN($t0, $t3, $t4)], host=[$t1], cpu_usage=[$t2], @timestamp0=[$t5]) + CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[@timestamp, host, cpu_usage], FILTER->AND(IS NOT NULL($0), IS NOT NULL($2))], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"@timestamp","boost":1.0}},{"exists":{"field":"cpu_usage","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["@timestamp","host","cpu_usage"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) EnumerableSort(sort0=[$0], dir0=[ASC]) - EnumerableLimit(fetch=[10]) - EnumerableSort(sort0=[$1], dir0=[DESC]) - EnumerableCalc(expr#0..2=[{inputs}], expr#3=[0], expr#4=[=($t2, $t3)], expr#5=[null:DOUBLE], expr#6=[CASE($t4, $t5, $t1)], expr#7=[/($t6, $t2)], host=[$t0], grand_total=[$t7]) - EnumerableAggregate(group=[{0}], agg#0=[$SUM0($1)], agg#1=[COUNT($1)]) - EnumerableCalc(expr#0..3=[{inputs}], expr#4=[0], expr#5=[=($t3, $t4)], expr#6=[null:DOUBLE], expr#7=[CASE($t5, $t6, $t2)], expr#8=[/($t7, $t3)], host=[$t0], $f2=[$t8]) - EnumerableAggregate(group=[{0, 2}], agg#0=[$SUM0($1)], agg#1=[COUNT($1)]) - EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], expr#4=['m'], expr#5=[SPAN($t2, $t3, $t4)], proj#0..1=[{exprs}], $f2=[$t5]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[host, cpu_usage, @timestamp], FILTER->IS NOT NULL($0)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"exists":{"field":"host","boost":1.0}},"_source":{"includes":["host","cpu_usage","@timestamp"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + EnumerableCalc(expr#0..2=[{inputs}], host=[$t0], $1=[$t2]) + EnumerableWindow(window#0=[window(order by [1 DESC-nulls-last] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) + EnumerableAggregate(group=[{0}], __grand_total__=[SUM($1)]) + EnumerableCalc(expr#0..3=[{inputs}], expr#4=[0], expr#5=[=($t3, $t4)], expr#6=[null:DOUBLE], expr#7=[CASE($t5, $t6, $t2)], expr#8=[/($t7, $t3)], host=[$t0], avg(cpu_usage)=[$t8]) + EnumerableAggregate(group=[{0, 2}], agg#0=[$SUM0($1)], agg#1=[COUNT($1)]) + EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], expr#4=['m'], expr#5=[SPAN($t0, $t3, $t4)], host=[$t1], cpu_usage=[$t2], @timestamp0=[$t5]) + CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[@timestamp, host, cpu_usage], FILTER->AND(IS NOT NULL($0), IS NOT NULL($2)), FILTER->IS NOT NULL($1)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"bool":{"filter":[{"bool":{"must":[{"exists":{"field":"@timestamp","boost":1.0}},{"exists":{"field":"cpu_usage","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},{"exists":{"field":"host","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["@timestamp","host","cpu_usage"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart_count.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart_count.yaml index adba9c12202..17d8b0f917b 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart_count.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart_count.yaml @@ -2,70 +2,35 @@ calcite: logical: | LogicalSystemLimit(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000], type=[QUERY_SIZE_LIMIT]) LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC]) - LogicalAggregate(group=[{0, 1}], count=[SUM($2)]) - LogicalUnion(all=[false]) - LogicalAggregate(group=[{0, 1}], actual_count=[SUM($2)]) - LogicalProject(@timestamp=[CAST($0):TIMESTAMP(0) NOT NULL], host=[CASE(IS NOT NULL($3), $1, CASE(IS NULL($1), null:NULL, 'OTHER'))], count=[$2]) - LogicalJoin(condition=[IS NOT DISTINCT FROM($1, $3)], joinType=[left]) - LogicalProject(@timestamp=[$1], host=[$0], $f2_0=[$2]) - LogicalAggregate(group=[{0, 1}], agg#0=[COUNT()]) - LogicalProject(host=[$4], $f2=[SPAN($1, 1, 'm')]) - CalciteLogicalIndexScan(table=[[OpenSearch, events]]) - LogicalSort(sort0=[$1], dir0=[DESC], fetch=[10]) - LogicalAggregate(group=[{1}], grand_total=[SUM($2)]) - LogicalFilter(condition=[IS NOT NULL($1)]) - LogicalProject(@timestamp=[$1], host=[$0], $f2_0=[$2]) - LogicalAggregate(group=[{0, 1}], agg#0=[COUNT()]) - LogicalProject(host=[$4], $f2=[SPAN($1, 1, 'm')]) - CalciteLogicalIndexScan(table=[[OpenSearch, events]]) - LogicalProject(@timestamp=[CAST($0):TIMESTAMP(0) NOT NULL], host=[$1], count=[0]) - LogicalJoin(condition=[true], joinType=[inner]) - LogicalAggregate(group=[{0}]) - LogicalProject(@timestamp=[$1]) - LogicalAggregate(group=[{0, 1}], agg#0=[COUNT()]) - LogicalProject(host=[$4], $f2=[SPAN($1, 1, 'm')]) - CalciteLogicalIndexScan(table=[[OpenSearch, events]]) - LogicalAggregate(group=[{0}]) - LogicalProject($f0=[CASE(IS NOT NULL($3), $1, CASE(IS NULL($1), null:NULL, 'OTHER'))]) - LogicalJoin(condition=[IS NOT DISTINCT FROM($1, $3)], joinType=[left]) - LogicalProject(@timestamp=[$1], host=[$0], $f2_0=[$2]) - LogicalAggregate(group=[{0, 1}], agg#0=[COUNT()]) - LogicalProject(host=[$4], $f2=[SPAN($1, 1, 'm')]) - CalciteLogicalIndexScan(table=[[OpenSearch, events]]) - LogicalSort(sort0=[$1], dir0=[DESC], fetch=[10]) - LogicalAggregate(group=[{1}], grand_total=[SUM($2)]) + LogicalAggregate(group=[{0, 1}], count()=[SUM($2)]) + LogicalProject(@timestamp=[$0], host=[CASE(IS NULL($1), 'NULL', <=($5, 10), $1, 'OTHER')], count()=[$2]) + LogicalJoin(condition=[=($1, $3)], joinType=[left]) + LogicalProject(@timestamp=[$1], host=[$0], count()=[$2]) + LogicalAggregate(group=[{0, 1}], count()=[COUNT()]) + LogicalProject(host=[$4], @timestamp0=[SPAN($1, 1, 'm')]) + LogicalFilter(condition=[IS NOT NULL($1)]) + CalciteLogicalIndexScan(table=[[OpenSearch, events]]) + LogicalProject(host=[$0], __grand_total__=[$1], _row_number_chart_=[ROW_NUMBER() OVER (ORDER BY $1 DESC NULLS LAST)]) + LogicalAggregate(group=[{0}], __grand_total__=[SUM($1)]) + LogicalFilter(condition=[IS NOT NULL($0)]) + LogicalProject(host=[$0], count()=[$2]) + LogicalAggregate(group=[{0, 1}], count()=[COUNT()]) + LogicalProject(host=[$4], @timestamp0=[SPAN($1, 1, 'm')]) LogicalFilter(condition=[IS NOT NULL($1)]) - LogicalProject(@timestamp=[$1], host=[$0], $f2_0=[$2]) - LogicalAggregate(group=[{0, 1}], agg#0=[COUNT()]) - LogicalProject(host=[$4], $f2=[SPAN($1, 1, 'm')]) - CalciteLogicalIndexScan(table=[[OpenSearch, events]]) + CalciteLogicalIndexScan(table=[[OpenSearch, events]]) physical: | EnumerableLimit(fetch=[10000]) EnumerableSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC]) - EnumerableAggregate(group=[{0, 1}], count=[$SUM0($2)]) - EnumerableUnion(all=[false]) - EnumerableAggregate(group=[{0, 1}], actual_count=[$SUM0($2)]) - EnumerableCalc(expr#0..4=[{inputs}], expr#5=[CAST($t0):TIMESTAMP(0) NOT NULL], expr#6=[IS NOT NULL($t3)], expr#7=[IS NULL($t1)], expr#8=[null:NULL], expr#9=['OTHER'], expr#10=[CASE($t7, $t8, $t9)], expr#11=[CASE($t6, $t1, $t10)], @timestamp=[$t5], host=[$t11], count=[$t2]) - EnumerableNestedLoopJoin(condition=[IS NOT DISTINCT FROM($1, $3)], joinType=[left]) - EnumerableCalc(expr#0..2=[{inputs}], @timestamp=[$t1], host=[$t0], $f2_0=[$t2]) - EnumerableAggregate(group=[{0, 1}], agg#0=[COUNT()]) - EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=['m'], expr#4=[SPAN($t1, $t2, $t3)], host=[$t0], $f1=[$t4]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[host, @timestamp]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["host","@timestamp"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) - EnumerableLimit(fetch=[10]) - EnumerableSort(sort0=[$1], dir0=[DESC]) - EnumerableAggregate(group=[{0}], grand_total=[COUNT()]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[host, @timestamp], FILTER->IS NOT NULL($0)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"exists":{"field":"host","boost":1.0}},"_source":{"includes":["host","@timestamp"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) - EnumerableCalc(expr#0..1=[{inputs}], expr#2=[CAST($t0):TIMESTAMP(0) NOT NULL], expr#3=[0], @timestamp=[$t2], host=[$t1], count=[$t3]) - EnumerableNestedLoopJoin(condition=[true], joinType=[inner]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={1})], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"$f2":{"date_histogram":{"field":"@timestamp","missing_bucket":false,"order":"asc","fixed_interval":"1m"}}}]}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) - EnumerableAggregate(group=[{0}]) - EnumerableCalc(expr#0..2=[{inputs}], expr#3=[IS NOT NULL($t1)], expr#4=[IS NULL($t0)], expr#5=[null:NULL], expr#6=['OTHER'], expr#7=[CASE($t4, $t5, $t6)], expr#8=[CASE($t3, $t0, $t7)], $f0=[$t8]) - EnumerableNestedLoopJoin(condition=[IS NOT DISTINCT FROM($0, $1)], joinType=[left]) - EnumerableCalc(expr#0..1=[{inputs}], host=[$t0]) - EnumerableAggregate(group=[{0, 1}]) - EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=['m'], expr#4=[SPAN($t1, $t2, $t3)], host=[$t0], $f1=[$t4]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[host, @timestamp]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["host","@timestamp"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) - EnumerableLimit(fetch=[10]) - EnumerableSort(sort0=[$1], dir0=[DESC]) - EnumerableAggregate(group=[{0}], grand_total=[COUNT()]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[host, @timestamp], FILTER->IS NOT NULL($0)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"exists":{"field":"host","boost":1.0}},"_source":{"includes":["host","@timestamp"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableAggregate(group=[{0, 1}], count()=[$SUM0($2)]) + EnumerableCalc(expr#0..4=[{inputs}], expr#5=[IS NULL($t1)], expr#6=['NULL'], expr#7=[10], expr#8=[<=($t4, $t7)], expr#9=['OTHER'], expr#10=[CASE($t5, $t6, $t8, $t1, $t9)], @timestamp=[$t0], host=[$t10], count()=[$t2]) + EnumerableMergeJoin(condition=[=($1, $3)], joinType=[left]) + EnumerableSort(sort0=[$1], dir0=[ASC]) + EnumerableCalc(expr#0..2=[{inputs}], @timestamp=[$t1], host=[$t0], count()=[$t2]) + EnumerableAggregate(group=[{0, 1}], count()=[COUNT()]) + EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=['m'], expr#4=[SPAN($t0, $t2, $t3)], host=[$t1], @timestamp0=[$t4]) + CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[@timestamp, host], FILTER->IS NOT NULL($0)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"exists":{"field":"@timestamp","boost":1.0}},"_source":{"includes":["@timestamp","host"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + EnumerableSort(sort0=[$0], dir0=[ASC]) + EnumerableCalc(expr#0..2=[{inputs}], host=[$t0], $1=[$t2]) + EnumerableWindow(window#0=[window(order by [1 DESC-nulls-last] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) + EnumerableAggregate(group=[{1}], __grand_total__=[COUNT()]) + CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[@timestamp, host], FILTER->IS NOT NULL($0), FILTER->IS NOT NULL($1)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"bool":{"filter":[{"exists":{"field":"@timestamp","boost":1.0}},{"exists":{"field":"host","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["@timestamp","host"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_timechart.yaml b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_timechart.yaml index 5aa55ca656b..e982ce038e2 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_timechart.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_timechart.yaml @@ -3,37 +3,38 @@ calcite: LogicalSystemLimit(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000], type=[QUERY_SIZE_LIMIT]) LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC]) LogicalAggregate(group=[{0, 1}], avg(cpu_usage)=[AVG($2)]) - LogicalProject(@timestamp=[$0], host=[CASE(IS NOT NULL($3), $1, CASE(IS NULL($1), null:NULL, 'OTHER'))], avg(cpu_usage)=[$2]) + LogicalProject(@timestamp=[$0], host=[CASE(IS NULL($1), 'NULL', <=($5, 10), $1, 'OTHER')], avg(cpu_usage)=[$2]) LogicalJoin(condition=[=($1, $3)], joinType=[left]) - LogicalProject(@timestamp=[$1], host=[$0], $f2=[$2]) - LogicalAggregate(group=[{0, 2}], agg#0=[AVG($1)]) - LogicalProject(host=[$4], cpu_usage=[$7], $f3=[SPAN($1, 1, 'm')]) - CalciteLogicalIndexScan(table=[[OpenSearch, events]]) - LogicalSort(sort0=[$1], dir0=[DESC], fetch=[10]) - LogicalAggregate(group=[{1}], grand_total=[AVG($2)]) - LogicalFilter(condition=[IS NOT NULL($1)]) - LogicalProject(@timestamp=[$1], host=[$0], $f2=[$2]) - LogicalAggregate(group=[{0, 2}], agg#0=[AVG($1)]) - LogicalProject(host=[$4], cpu_usage=[$7], $f3=[SPAN($1, 1, 'm')]) - CalciteLogicalIndexScan(table=[[OpenSearch, events]]) + LogicalProject(@timestamp=[$1], host=[$0], avg(cpu_usage)=[$2]) + LogicalAggregate(group=[{0, 2}], avg(cpu_usage)=[AVG($1)]) + LogicalProject(host=[$4], cpu_usage=[$7], @timestamp0=[SPAN($1, 1, 'm')]) + LogicalFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($7))]) + CalciteLogicalIndexScan(table=[[OpenSearch, events]]) + LogicalProject(host=[$0], __grand_total__=[$1], _row_number_chart_=[ROW_NUMBER() OVER (ORDER BY $1 DESC NULLS LAST)]) + LogicalAggregate(group=[{0}], __grand_total__=[SUM($1)]) + LogicalFilter(condition=[IS NOT NULL($0)]) + LogicalProject(host=[$0], avg(cpu_usage)=[$2]) + LogicalAggregate(group=[{0, 2}], avg(cpu_usage)=[AVG($1)]) + LogicalProject(host=[$4], cpu_usage=[$7], @timestamp0=[SPAN($1, 1, 'm')]) + LogicalFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($7))]) + CalciteLogicalIndexScan(table=[[OpenSearch, events]]) physical: | EnumerableLimit(fetch=[10000]) EnumerableSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC]) EnumerableCalc(expr#0..3=[{inputs}], expr#4=[0], expr#5=[=($t3, $t4)], expr#6=[null:DOUBLE], expr#7=[CASE($t5, $t6, $t2)], expr#8=[/($t7, $t3)], proj#0..1=[{exprs}], avg(cpu_usage)=[$t8]) EnumerableAggregate(group=[{0, 1}], agg#0=[$SUM0($2)], agg#1=[COUNT($2)]) - EnumerableCalc(expr#0..4=[{inputs}], expr#5=[IS NOT NULL($t3)], expr#6=[IS NULL($t1)], expr#7=[null:NULL], expr#8=['OTHER'], expr#9=[CASE($t6, $t7, $t8)], expr#10=[CASE($t5, $t1, $t9)], @timestamp=[$t0], host=[$t10], avg(cpu_usage)=[$t2]) + EnumerableCalc(expr#0..4=[{inputs}], expr#5=[IS NULL($t1)], expr#6=['NULL'], expr#7=[10], expr#8=[<=($t4, $t7)], expr#9=['OTHER'], expr#10=[CASE($t5, $t6, $t8, $t1, $t9)], @timestamp=[$t0], host=[$t10], avg(cpu_usage)=[$t2]) EnumerableMergeJoin(condition=[=($1, $3)], joinType=[left]) EnumerableSort(sort0=[$1], dir0=[ASC]) - EnumerableCalc(expr#0..3=[{inputs}], expr#4=[0], expr#5=[=($t3, $t4)], expr#6=[null:DOUBLE], expr#7=[CASE($t5, $t6, $t2)], expr#8=[/($t7, $t3)], @timestamp=[$t1], host=[$t0], $f2=[$t8]) + EnumerableCalc(expr#0..3=[{inputs}], expr#4=[0], expr#5=[=($t3, $t4)], expr#6=[null:DOUBLE], expr#7=[CASE($t5, $t6, $t2)], expr#8=[/($t7, $t3)], @timestamp=[$t1], host=[$t0], avg(cpu_usage)=[$t8]) EnumerableAggregate(group=[{0, 2}], agg#0=[$SUM0($1)], agg#1=[COUNT($1)]) - EnumerableCalc(expr#0..15=[{inputs}], expr#16=[1], expr#17=['m'], expr#18=[SPAN($t1, $t16, $t17)], host=[$t4], cpu_usage=[$t7], $f3=[$t18]) + EnumerableCalc(expr#0..15=[{inputs}], expr#16=[1], expr#17=['m'], expr#18=[SPAN($t1, $t16, $t17)], expr#19=[IS NOT NULL($t1)], expr#20=[IS NOT NULL($t7)], expr#21=[AND($t19, $t20)], host=[$t4], cpu_usage=[$t7], @timestamp0=[$t18], $condition=[$t21]) CalciteEnumerableIndexScan(table=[[OpenSearch, events]]) EnumerableSort(sort0=[$0], dir0=[ASC]) - EnumerableLimit(fetch=[10]) - EnumerableSort(sort0=[$1], dir0=[DESC]) - EnumerableCalc(expr#0..2=[{inputs}], expr#3=[0], expr#4=[=($t2, $t3)], expr#5=[null:DOUBLE], expr#6=[CASE($t4, $t5, $t1)], expr#7=[/($t6, $t2)], host=[$t0], grand_total=[$t7]) - EnumerableAggregate(group=[{0}], agg#0=[$SUM0($2)], agg#1=[COUNT($2)]) - EnumerableCalc(expr#0..3=[{inputs}], expr#4=[0], expr#5=[=($t3, $t4)], expr#6=[null:DOUBLE], expr#7=[CASE($t5, $t6, $t2)], expr#8=[/($t7, $t3)], expr#9=[IS NOT NULL($t0)], proj#0..1=[{exprs}], $f2=[$t8], $condition=[$t9]) - EnumerableAggregate(group=[{0, 2}], agg#0=[$SUM0($1)], agg#1=[COUNT($1)]) - EnumerableCalc(expr#0..15=[{inputs}], expr#16=[1], expr#17=['m'], expr#18=[SPAN($t1, $t16, $t17)], host=[$t4], cpu_usage=[$t7], $f3=[$t18]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]]) + EnumerableCalc(expr#0..2=[{inputs}], host=[$t0], $1=[$t2]) + EnumerableWindow(window#0=[window(order by [1 DESC-nulls-last] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) + EnumerableAggregate(group=[{0}], __grand_total__=[SUM($2)]) + EnumerableCalc(expr#0..3=[{inputs}], expr#4=[0], expr#5=[=($t3, $t4)], expr#6=[null:DOUBLE], expr#7=[CASE($t5, $t6, $t2)], expr#8=[/($t7, $t3)], proj#0..1=[{exprs}], avg(cpu_usage)=[$t8]) + EnumerableAggregate(group=[{0, 2}], agg#0=[$SUM0($1)], agg#1=[COUNT($1)]) + EnumerableCalc(expr#0..15=[{inputs}], expr#16=[1], expr#17=['m'], expr#18=[SPAN($t1, $t16, $t17)], expr#19=[IS NOT NULL($t1)], expr#20=[IS NOT NULL($t7)], expr#21=[IS NOT NULL($t4)], expr#22=[AND($t19, $t20, $t21)], host=[$t4], cpu_usage=[$t7], @timestamp0=[$t18], $condition=[$t22]) + CalciteEnumerableIndexScan(table=[[OpenSearch, events]]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_timechart_count.yaml b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_timechart_count.yaml index e60799af17d..2979778506a 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_timechart_count.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_timechart_count.yaml @@ -2,74 +2,36 @@ calcite: logical: | LogicalSystemLimit(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000], type=[QUERY_SIZE_LIMIT]) LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC]) - LogicalAggregate(group=[{0, 1}], count=[SUM($2)]) - LogicalUnion(all=[false]) - LogicalAggregate(group=[{0, 1}], actual_count=[SUM($2)]) - LogicalProject(@timestamp=[CAST($0):TIMESTAMP(0) NOT NULL], host=[CASE(IS NOT NULL($3), $1, CASE(IS NULL($1), null:NULL, 'OTHER'))], count=[$2]) - LogicalJoin(condition=[IS NOT DISTINCT FROM($1, $3)], joinType=[left]) - LogicalProject(@timestamp=[$1], host=[$0], $f2_0=[$2]) - LogicalAggregate(group=[{0, 1}], agg#0=[COUNT()]) - LogicalProject(host=[$4], $f2=[SPAN($1, 1, 'm')]) - CalciteLogicalIndexScan(table=[[OpenSearch, events]]) - LogicalSort(sort0=[$1], dir0=[DESC], fetch=[10]) - LogicalAggregate(group=[{1}], grand_total=[SUM($2)]) - LogicalFilter(condition=[IS NOT NULL($1)]) - LogicalProject(@timestamp=[$1], host=[$0], $f2_0=[$2]) - LogicalAggregate(group=[{0, 1}], agg#0=[COUNT()]) - LogicalProject(host=[$4], $f2=[SPAN($1, 1, 'm')]) - CalciteLogicalIndexScan(table=[[OpenSearch, events]]) - LogicalProject(@timestamp=[CAST($0):TIMESTAMP(0) NOT NULL], host=[$1], count=[0]) - LogicalJoin(condition=[true], joinType=[inner]) - LogicalAggregate(group=[{0}]) - LogicalProject(@timestamp=[$1]) - LogicalAggregate(group=[{0, 1}], agg#0=[COUNT()]) - LogicalProject(host=[$4], $f2=[SPAN($1, 1, 'm')]) - CalciteLogicalIndexScan(table=[[OpenSearch, events]]) - LogicalAggregate(group=[{0}]) - LogicalProject($f0=[CASE(IS NOT NULL($3), $1, CASE(IS NULL($1), null:NULL, 'OTHER'))]) - LogicalJoin(condition=[IS NOT DISTINCT FROM($1, $3)], joinType=[left]) - LogicalProject(@timestamp=[$1], host=[$0], $f2_0=[$2]) - LogicalAggregate(group=[{0, 1}], agg#0=[COUNT()]) - LogicalProject(host=[$4], $f2=[SPAN($1, 1, 'm')]) - CalciteLogicalIndexScan(table=[[OpenSearch, events]]) - LogicalSort(sort0=[$1], dir0=[DESC], fetch=[10]) - LogicalAggregate(group=[{1}], grand_total=[SUM($2)]) + LogicalAggregate(group=[{0, 1}], count()=[SUM($2)]) + LogicalProject(@timestamp=[$0], host=[CASE(IS NULL($1), 'NULL', <=($5, 10), $1, 'OTHER')], count()=[$2]) + LogicalJoin(condition=[=($1, $3)], joinType=[left]) + LogicalProject(@timestamp=[$1], host=[$0], count()=[$2]) + LogicalAggregate(group=[{0, 1}], count()=[COUNT()]) + LogicalProject(host=[$4], @timestamp0=[SPAN($1, 1, 'm')]) + LogicalFilter(condition=[IS NOT NULL($1)]) + CalciteLogicalIndexScan(table=[[OpenSearch, events]]) + LogicalProject(host=[$0], __grand_total__=[$1], _row_number_chart_=[ROW_NUMBER() OVER (ORDER BY $1 DESC NULLS LAST)]) + LogicalAggregate(group=[{0}], __grand_total__=[SUM($1)]) + LogicalFilter(condition=[IS NOT NULL($0)]) + LogicalProject(host=[$0], count()=[$2]) + LogicalAggregate(group=[{0, 1}], count()=[COUNT()]) + LogicalProject(host=[$4], @timestamp0=[SPAN($1, 1, 'm')]) LogicalFilter(condition=[IS NOT NULL($1)]) - LogicalProject(@timestamp=[$1], host=[$0], $f2_0=[$2]) - LogicalAggregate(group=[{0, 1}], agg#0=[COUNT()]) - LogicalProject(host=[$4], $f2=[SPAN($1, 1, 'm')]) - CalciteLogicalIndexScan(table=[[OpenSearch, events]]) + CalciteLogicalIndexScan(table=[[OpenSearch, events]]) physical: | EnumerableLimit(fetch=[10000]) EnumerableSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC]) - EnumerableAggregate(group=[{0, 1}], count=[$SUM0($2)]) - EnumerableUnion(all=[false]) - EnumerableAggregate(group=[{0, 1}], actual_count=[$SUM0($2)]) - EnumerableCalc(expr#0..4=[{inputs}], expr#5=[CAST($t0):TIMESTAMP(0) NOT NULL], expr#6=[IS NOT NULL($t3)], expr#7=[IS NULL($t1)], expr#8=[null:NULL], expr#9=['OTHER'], expr#10=[CASE($t7, $t8, $t9)], expr#11=[CASE($t6, $t1, $t10)], @timestamp=[$t5], host=[$t11], count=[$t2]) - EnumerableNestedLoopJoin(condition=[IS NOT DISTINCT FROM($1, $3)], joinType=[left]) - EnumerableCalc(expr#0..2=[{inputs}], @timestamp=[$t1], host=[$t0], $f2_0=[$t2]) - EnumerableAggregate(group=[{0, 1}], agg#0=[COUNT()]) - EnumerableCalc(expr#0..15=[{inputs}], expr#16=[1], expr#17=['m'], expr#18=[SPAN($t1, $t16, $t17)], host=[$t4], $f2=[$t18]) + EnumerableAggregate(group=[{0, 1}], count()=[$SUM0($2)]) + EnumerableCalc(expr#0..4=[{inputs}], expr#5=[IS NULL($t1)], expr#6=['NULL'], expr#7=[10], expr#8=[<=($t4, $t7)], expr#9=['OTHER'], expr#10=[CASE($t5, $t6, $t8, $t1, $t9)], @timestamp=[$t0], host=[$t10], count()=[$t2]) + EnumerableMergeJoin(condition=[=($1, $3)], joinType=[left]) + EnumerableSort(sort0=[$1], dir0=[ASC]) + EnumerableCalc(expr#0..2=[{inputs}], @timestamp=[$t1], host=[$t0], count()=[$t2]) + EnumerableAggregate(group=[{0, 1}], count()=[COUNT()]) + EnumerableCalc(expr#0..15=[{inputs}], expr#16=[1], expr#17=['m'], expr#18=[SPAN($t1, $t16, $t17)], expr#19=[IS NOT NULL($t1)], host=[$t4], @timestamp0=[$t18], $condition=[$t19]) + CalciteEnumerableIndexScan(table=[[OpenSearch, events]]) + EnumerableSort(sort0=[$0], dir0=[ASC]) + EnumerableCalc(expr#0..2=[{inputs}], host=[$t0], $1=[$t2]) + EnumerableWindow(window#0=[window(order by [1 DESC-nulls-last] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) + EnumerableAggregate(group=[{4}], __grand_total__=[COUNT()]) + EnumerableCalc(expr#0..15=[{inputs}], expr#16=[IS NOT NULL($t1)], expr#17=[IS NOT NULL($t4)], expr#18=[AND($t16, $t17)], proj#0..15=[{exprs}], $condition=[$t18]) CalciteEnumerableIndexScan(table=[[OpenSearch, events]]) - EnumerableLimit(fetch=[10]) - EnumerableSort(sort0=[$1], dir0=[DESC]) - EnumerableAggregate(group=[{4}], grand_total=[COUNT()]) - EnumerableCalc(expr#0..15=[{inputs}], expr#16=[IS NOT NULL($t4)], proj#0..15=[{exprs}], $condition=[$t16]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]]) - EnumerableCalc(expr#0..1=[{inputs}], expr#2=[CAST($t0):TIMESTAMP(0) NOT NULL], expr#3=[0], @timestamp=[$t2], host=[$t1], count=[$t3]) - EnumerableNestedLoopJoin(condition=[true], joinType=[inner]) - EnumerableAggregate(group=[{1}]) - EnumerableCalc(expr#0..15=[{inputs}], expr#16=[1], expr#17=['m'], expr#18=[SPAN($t1, $t16, $t17)], host=[$t4], $f2=[$t18]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]]) - EnumerableAggregate(group=[{0}]) - EnumerableCalc(expr#0..2=[{inputs}], expr#3=[IS NOT NULL($t1)], expr#4=[IS NULL($t0)], expr#5=[null:NULL], expr#6=['OTHER'], expr#7=[CASE($t4, $t5, $t6)], expr#8=[CASE($t3, $t0, $t7)], $f0=[$t8]) - EnumerableNestedLoopJoin(condition=[IS NOT DISTINCT FROM($0, $1)], joinType=[left]) - EnumerableCalc(expr#0..1=[{inputs}], host=[$t0]) - EnumerableAggregate(group=[{0, 1}]) - EnumerableCalc(expr#0..15=[{inputs}], expr#16=[1], expr#17=['m'], expr#18=[SPAN($t1, $t16, $t17)], host=[$t4], $f2=[$t18]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]]) - EnumerableLimit(fetch=[10]) - EnumerableSort(sort0=[$1], dir0=[DESC]) - EnumerableAggregate(group=[{4}], grand_total=[COUNT()]) - EnumerableCalc(expr#0..15=[{inputs}], expr#16=[IS NOT NULL($t4)], proj#0..15=[{exprs}], $condition=[$t16]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]]) \ No newline at end of file diff --git a/integ-test/src/yamlRestTest/resources/rest-api-spec/test/issues/4550.yml b/integ-test/src/yamlRestTest/resources/rest-api-spec/test/issues/4550.yml index 147ab899552..06ce51afbc8 100644 --- a/integ-test/src/yamlRestTest/resources/rest-api-spec/test/issues/4550.yml +++ b/integ-test/src/yamlRestTest/resources/rest-api-spec/test/issues/4550.yml @@ -41,7 +41,7 @@ setup: query: source=test_data_2023 | timechart span=500ms count() - match: { total: 5 } - - match: { "schema": [ { "name": "@timestamp", "type": "timestamp" }, { "name": "count", "type": "bigint" }] } + - match: { "schema": [ { "name": "@timestamp", "type": "timestamp" }, { "name": "count()", "type": "bigint" }] } - match: {"datarows": [["2023-10-08 10:00:00", 1], ["2023-10-08 10:00:00.5", 1], ["2023-10-08 10:00:01", 1], ["2023-10-08 10:00:01.5", 1], ["2023-10-08 10:00:02", 1]]} --- @@ -79,7 +79,7 @@ setup: query: source=test_data_2023 | timechart span=250milliseconds count() - match: { total: 5 } - - match: { "schema": [ { "name": "@timestamp", "type": "timestamp" }, { "name": "count", "type": "bigint" }] } + - match: { "schema": [ { "name": "@timestamp", "type": "timestamp" }, { "name": "count()", "type": "bigint" }] } --- "timechart with second span for comparison": @@ -97,5 +97,5 @@ setup: query: source=test_data_2023 | timechart span=1s count() - match: { total: 3 } - - match: { "schema": [ { "name": "@timestamp", "type": "timestamp" }, { "name": "count", "type": "bigint" }] } + - match: { "schema": [ { "name": "@timestamp", "type": "timestamp" }, { "name": "count()", "type": "bigint" }] } - match: {"datarows": [["2023-10-08 10:00:00", 2], ["2023-10-08 10:00:01", 2], ["2023-10-08 10:00:02", 1]]} diff --git a/integ-test/src/yamlRestTest/resources/rest-api-spec/test/issues/4581.yml b/integ-test/src/yamlRestTest/resources/rest-api-spec/test/issues/4581.yml new file mode 100644 index 00000000000..3ef89e9f90c --- /dev/null +++ b/integ-test/src/yamlRestTest/resources/rest-api-spec/test/issues/4581.yml @@ -0,0 +1,53 @@ +setup: + - do: + indices.create: + index: opensearch-sql_test_index_otel_logs + body: + mappings: + properties: + "@timestamp": + type: date + "severityNumber": + type: integer + "flags": + type: long + - do: + bulk: + index: opensearch-sql_test_index_otel_logs + refresh: true + body: + - '{"index":{}}' + - '{"@timestamp":"2023-10-08T10:00:00.000Z","severityNumber":10,"flags":0}' + - '{"index":{}}' + - '{"@timestamp":"2023-10-08T10:00:01.000Z","severityNumber":12,"flags":0}' + - '{"index":{}}' + - '{"@timestamp":"2023-10-08T10:00:02.000Z","severityNumber":15,"flags":0}' + - '{"index":{}}' + - '{"@timestamp":"2023-10-08T10:00:03.000Z","severityNumber":13,"flags":0}' + - '{"index":{}}' + - '{"@timestamp":"2023-10-08T10:00:04.000Z","severityNumber":11,"flags":0}' + - '{"index":{}}' + - '{"@timestamp":"2023-10-08T10:00:05.000Z","severityNumber":14,"flags":0}' + - '{"index":{}}' + - '{"@timestamp":"2023-10-08T10:00:06.000Z","severityNumber":16,"flags":0}' + - '{"index":{}}' + - '{"@timestamp":"2023-10-08T10:00:07.000Z","severityNumber":9,"flags":1}' + +--- +"timechart with grouping by non-string field": + - skip: + features: + - headers + - allowed_warnings + - do: + allowed_warnings: + - 'Loading the fielddata on the _id field is deprecated and will be removed in future versions. If you require sorting or aggregating on this field you should also include the id in the body of your documents, and map this field as a keyword field that has [doc_values] enabled' + headers: + Content-Type: 'application/json' + ppl: + body: + query: source=opensearch-sql_test_index_otel_logs | timechart avg(severityNumber) by flags + + - match: { total: 2 } + - match: { "schema": [ { "name": "@timestamp", "type": "timestamp" }, { "name": "flags", "type": "string" }, { "name": "avg(severityNumber)", "type": "double" }] } + - match: {"datarows": [["2023-10-08 10:00:00", "0", 13.0], ["2023-10-08 10:00:00", "1", 9.0]]} diff --git a/integ-test/src/yamlRestTest/resources/rest-api-spec/test/issues/4582.yml b/integ-test/src/yamlRestTest/resources/rest-api-spec/test/issues/4582.yml new file mode 100644 index 00000000000..952c4cd6387 --- /dev/null +++ b/integ-test/src/yamlRestTest/resources/rest-api-spec/test/issues/4582.yml @@ -0,0 +1,52 @@ +setup: + - do: + indices.create: + index: opensearch-sql_test_index_otel_logs + body: + mappings: + properties: + "@timestamp": + type: date + "severityNumber": + type: long + "severityText": + type: keyword + - do: + bulk: + index: opensearch-sql_test_index_otel_logs + refresh: true + body: + - '{"index":{}}' + - '{"@timestamp":"2024-01-15T10:00:00.000Z","severityNumber":24,"severityText":"FATAL4"}' + - '{"index":{}}' + - '{"@timestamp":"2024-01-15T10:00:01.000Z","severityNumber":24,"severityText":"FATAL4"}' + - '{"index":{}}' + - '{"@timestamp":"2024-01-15T10:00:02.000Z","severityNumber":23,"severityText":"ERROR5"}' + - '{"index":{}}' + - '{"@timestamp":"2024-01-15T10:00:03.000Z","severityNumber":22,"severityText":"ERROR6"}' + - '{"index":{}}' + - '{"@timestamp":"2024-01-15T10:00:04.000Z","severityNumber":21,"severityText":"WARN7"}' + - '{"index":{}}' + - '{"@timestamp":"2024-01-15T10:00:05.000Z","severityNumber":20,"severityText":"INFO8"}' + - '{"index":{}}' + - '{"@timestamp":"2024-01-15T10:00:06.000Z","severityNumber":19,"severityText":"DEBUG9"}' + - '{"index":{}}' + - '{"@timestamp":"2024-01-15T10:00:07.000Z","severityNumber":18,"severityText":"TRACE10"}' + +--- +"timechart with limit should use max aggregation for OTHER category": + - skip: + features: + - headers + - do: + headers: + Content-Type: 'application/json' + ppl: + body: + query: source=opensearch-sql_test_index_otel_logs | timechart limit=1 span=1d max(severityNumber) by severityText + + - match: { total: 2 } + - match: { "schema": [ { "name": "@timestamp", "type": "timestamp" }, { "name": "severityText", "type": "string" }, { "name": "max(severityNumber)", "type": "bigint" }] } + # The FATAL4 category has max(severityNumber) = 24 + # The OTHER category should have max(severityNumber) = 23 (not sum of all others which would be 143) + - match: {"datarows": [["2024-01-15 00:00:00", "FATAL4", 24], ["2024-01-15 00:00:00", "OTHER", 23]]} diff --git a/integ-test/src/yamlRestTest/resources/rest-api-spec/test/issues/4632.yml b/integ-test/src/yamlRestTest/resources/rest-api-spec/test/issues/4632.yml new file mode 100644 index 00000000000..9fad86ff2e7 --- /dev/null +++ b/integ-test/src/yamlRestTest/resources/rest-api-spec/test/issues/4632.yml @@ -0,0 +1,69 @@ +setup: + - do: + indices.create: + index: events + body: + mappings: + properties: + "@timestamp": + type: date + "host": + type: text + "packets": + type: long + - do: + bulk: + index: events + refresh: true + body: + - '{"index":{}}' + - '{"@timestamp":"2023-01-01T10:00:00Z","host":"server1","packets":60}' + - '{"index":{}}' + - '{"@timestamp":"2023-01-01T10:05:00Z","host":"server2","packets":30}' + - '{"index":{}}' + - '{"@timestamp":"2023-01-01T10:10:00Z","host":"server1","packets":60}' + - '{"index":{}}' + - '{"@timestamp":"2023-01-01T10:15:00Z","host":"server2","packets":30}' + - '{"index":{}}' + - '{"@timestamp":"2023-01-01T10:20:00Z","host":"server1","packets":60}' + - '{"index":{}}' + - '{"@timestamp":"2023-01-01T10:25:00Z","host":"server2","packets":30}' + - '{"index":{}}' + - '{"@timestamp":"2023-01-01T10:30:00Z","host":"server1","packets":180}' + - '{"index":{}}' + - '{"@timestamp":"2023-01-01T10:35:00Z","host":"server2","packets":90}' + +--- +"timechart count() by field should not return empty buckets": + - skip: + features: + - headers + - do: + headers: + Content-Type: 'application/json' + ppl: + body: + query: source=events | timechart span=1m count() by host + + - match: { total: 8 } + - match: { "schema": [ { "name": "@timestamp", "type": "timestamp" }, { "name": "host", "type": "string" }, { "name": "count()", "type": "bigint" }] } + # Verify that only non-empty buckets are returned (8 rows instead of 16) + # Each time bucket should only have one host with actual data + - match: {"datarows": [["2023-01-01 10:00:00", "server1", 1], ["2023-01-01 10:05:00", "server2", 1], ["2023-01-01 10:10:00", "server1", 1], ["2023-01-01 10:15:00", "server2", 1], ["2023-01-01 10:20:00", "server1", 1], ["2023-01-01 10:25:00", "server2", 1], ["2023-01-01 10:30:00", "server1", 1], ["2023-01-01 10:35:00", "server2", 1]]} + +--- +"timechart max() by field should not return empty buckets": + - skip: + features: + - headers + - do: + headers: + Content-Type: 'application/json' + ppl: + body: + query: source=events | timechart span=1m max(packets) by host + + - match: { total: 8 } + - match: { "schema": [ { "name": "@timestamp", "type": "timestamp" }, { "name": "host", "type": "string" }, { "name": "max(packets)", "type": "bigint" }] } + # Verify that only non-empty buckets are returned + - match: {"datarows": [["2023-01-01 10:00:00", "server1", 60], ["2023-01-01 10:05:00", "server2", 30], ["2023-01-01 10:10:00", "server1", 60], ["2023-01-01 10:15:00", "server2", 30], ["2023-01-01 10:20:00", "server1", 60], ["2023-01-01 10:25:00", "server2", 30], ["2023-01-01 10:30:00", "server1", 180], ["2023-01-01 10:35:00", "server2", 90]]} diff --git a/ppl/src/main/antlr/OpenSearchPPLParser.g4 b/ppl/src/main/antlr/OpenSearchPPLParser.g4 index 3857c8557b0..4edccd708ed 100644 --- a/ppl/src/main/antlr/OpenSearchPPLParser.g4 +++ b/ppl/src/main/antlr/OpenSearchPPLParser.g4 @@ -289,8 +289,8 @@ reverseCommand ; chartCommand - : CHART chartOptions* statsAggTerm (OVER rowSplit)? (BY columnSplit)? - | CHART chartOptions* statsAggTerm BY rowSplit (COMMA)? columnSplit + : CHART chartOptions* statsAggTerm (OVER rowSplit)? (BY columnSplit)? chartOptions* + | CHART chartOptions* statsAggTerm BY rowSplit (COMMA)? columnSplit chartOptions* ; chartOptions @@ -311,7 +311,7 @@ columnSplit ; timechartCommand - : TIMECHART timechartParameter* statsFunction (BY fieldExpression)? + : TIMECHART timechartParameter* statsAggTerm (BY fieldExpression)? timechartParameter* ; timechartParameter diff --git a/ppl/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java b/ppl/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java index 09e9b4c77ed..34e60951c3c 100644 --- a/ppl/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java +++ b/ppl/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java @@ -110,10 +110,10 @@ import org.opensearch.sql.ast.tree.StreamWindow; import org.opensearch.sql.ast.tree.SubqueryAlias; import org.opensearch.sql.ast.tree.TableFunction; -import org.opensearch.sql.ast.tree.Timechart; import org.opensearch.sql.ast.tree.Trendline; import org.opensearch.sql.ast.tree.UnresolvedPlan; import org.opensearch.sql.ast.tree.Window; +import org.opensearch.sql.calcite.plan.OpenSearchConstants; import org.opensearch.sql.common.antlr.SyntaxCheckException; import org.opensearch.sql.common.setting.Settings; import org.opensearch.sql.common.setting.Settings.Key; @@ -744,7 +744,7 @@ private List parseAggTerms( @Override public UnresolvedPlan visitTimechartCommand(OpenSearchPPLParser.TimechartCommandContext ctx) { UnresolvedExpression binExpression = - AstDSL.span(AstDSL.referImplicitTimestampField(), AstDSL.intLiteral(1), SpanUnit.m); + AstDSL.span(AstDSL.implicitTimestampField(), AstDSL.intLiteral(1), SpanUnit.m); Integer limit = 10; Boolean useOther = true; // Process timechart parameters @@ -761,16 +761,26 @@ public UnresolvedPlan visitTimechartCommand(OpenSearchPPLParser.TimechartCommand } } } + UnresolvedExpression aggregateFunction = parseAggTerms(List.of(ctx.statsAggTerm())).getFirst(); - UnresolvedExpression aggregateFunction = internalVisitExpression(ctx.statsFunction()); UnresolvedExpression byField = ctx.fieldExpression() != null ? internalVisitExpression(ctx.fieldExpression()) : null; - - return new Timechart(null, aggregateFunction) - .span(binExpression) - .by(byField) - .limit(limit) - .useOther(useOther); + List arguments = + List.of( + new Argument("limit", AstDSL.intLiteral(limit)), + new Argument("useother", AstDSL.booleanLiteral(useOther))); + binExpression = AstDSL.alias(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP, binExpression); + if (byField != null) { + byField = + AstDSL.alias( + StringUtils.unquoteIdentifier(getTextInQuery(ctx.fieldExpression())), byField); + } + return Chart.builder() + .aggregationFunction(aggregateFunction) + .rowSplit(binExpression) + .columnSplit(byField) + .arguments(arguments) + .build(); } /** Eval command. */ diff --git a/ppl/src/main/java/org/opensearch/sql/ppl/parser/AstExpressionBuilder.java b/ppl/src/main/java/org/opensearch/sql/ppl/parser/AstExpressionBuilder.java index 5dc1bf44d86..16f37e28d8c 100644 --- a/ppl/src/main/java/org/opensearch/sql/ppl/parser/AstExpressionBuilder.java +++ b/ppl/src/main/java/org/opensearch/sql/ppl/parser/AstExpressionBuilder.java @@ -69,7 +69,6 @@ import org.opensearch.sql.ppl.antlr.parser.OpenSearchPPLParser.StatsFunctionCallContext; import org.opensearch.sql.ppl.antlr.parser.OpenSearchPPLParser.StringLiteralContext; import org.opensearch.sql.ppl.antlr.parser.OpenSearchPPLParser.TableSourceContext; -import org.opensearch.sql.ppl.antlr.parser.OpenSearchPPLParser.TimechartCommandContext; import org.opensearch.sql.ppl.antlr.parser.OpenSearchPPLParser.WcFieldExpressionContext; import org.opensearch.sql.ppl.antlr.parser.OpenSearchPPLParserBaseVisitor; import org.opensearch.sql.ppl.utils.ArgumentFactory; @@ -586,9 +585,18 @@ private List timestampFunctionArguments( @Override public UnresolvedExpression visitPerFunctionCall(PerFunctionCallContext ctx) { - ParseTree parent = ctx.getParent(); String perFuncName = ctx.perFunction().funcName.getText(); - if (!(parent instanceof TimechartCommandContext)) { + // Walk up the parent tree to find timechart command context + ParseTree current = ctx.getParent(); + boolean foundTimechartContext = false; + while (current != null) { + if (current instanceof OpenSearchPPLParser.TimechartCommandContext) { + foundTimechartContext = true; + break; + } + current = current.getParent(); + } + if (!foundTimechartContext) { throw new SyntaxCheckException( perFuncName + " function can only be used within timechart command"); } @@ -670,7 +678,7 @@ public UnresolvedExpression visitSpanClause(SpanClauseContext ctx) { if (ctx.fieldExpression() != null) { fieldExpression = visit(ctx.fieldExpression()); } else { - fieldExpression = AstDSL.referImplicitTimestampField(); + fieldExpression = AstDSL.implicitTimestampField(); } Literal literal = (Literal) visit(ctx.value); return AstDSL.spanFromSpanLengthLiteral(fieldExpression, literal); @@ -976,7 +984,7 @@ public UnresolvedExpression visitTimechartParameter( // Convert span=1h to span(@timestamp, 1h) Literal spanLiteral = (Literal) visit(ctx.spanLiteral()); timechartParameter = - AstDSL.spanFromSpanLengthLiteral(AstDSL.referImplicitTimestampField(), spanLiteral); + AstDSL.spanFromSpanLengthLiteral(AstDSL.implicitTimestampField(), spanLiteral); } else if (ctx.LIMIT() != null) { Literal limit = (Literal) visit(ctx.integerLiteral()); if ((Integer) limit.getValue() < 0) { diff --git a/ppl/src/main/java/org/opensearch/sql/ppl/utils/PPLQueryDataAnonymizer.java b/ppl/src/main/java/org/opensearch/sql/ppl/utils/PPLQueryDataAnonymizer.java index 0971924295a..d2dd8d9955c 100644 --- a/ppl/src/main/java/org/opensearch/sql/ppl/utils/PPLQueryDataAnonymizer.java +++ b/ppl/src/main/java/org/opensearch/sql/ppl/utils/PPLQueryDataAnonymizer.java @@ -89,7 +89,6 @@ import org.opensearch.sql.ast.tree.StreamWindow; import org.opensearch.sql.ast.tree.SubqueryAlias; import org.opensearch.sql.ast.tree.TableFunction; -import org.opensearch.sql.ast.tree.Timechart; import org.opensearch.sql.ast.tree.Trendline; import org.opensearch.sql.ast.tree.UnresolvedPlan; import org.opensearch.sql.ast.tree.Values; @@ -500,43 +499,14 @@ public String visitReverse(Reverse node, String context) { return StringUtils.format("%s | reverse", child); } - @Override - public String visitTimechart(Timechart node, String context) { - String child = node.getChild().get(0).accept(this, context); - StringBuilder timechartCommand = new StringBuilder(); - timechartCommand.append(" | timechart"); - - // Add span if present - if (node.getBinExpression() != null) { - timechartCommand.append(" span=").append(visitExpression(node.getBinExpression())); - } - - // Add limit if present - if (node.getLimit() != null) { - timechartCommand.append(" limit=").append(node.getLimit()); - } - - // Add useother if present - if (node.getUseOther() != null) { - timechartCommand.append(" useother=").append(node.getUseOther()); - } - - // Add aggregation function - timechartCommand.append(" ").append(visitExpression(node.getAggregateFunction())); - - // Add by clause if present - if (node.getByField() != null) { - timechartCommand.append(" by ").append(visitExpression(node.getByField())); - } - - return StringUtils.format("%s%s", child, timechartCommand.toString()); - } - @Override public String visitChart(Chart node, String context) { String child = node.getChild().get(0).accept(this, context); StringBuilder chartCommand = new StringBuilder(); - chartCommand.append(" | chart"); + + // Check if this is a timechart by looking for timestamp span in rowSplit + boolean isTimechart = isTimechartNode(node); + chartCommand.append(isTimechart ? " | timechart" : " | chart"); for (Argument arg : node.getArguments()) { String argName = arg.getArgName(); @@ -568,6 +538,20 @@ public String visitChart(Chart node, String context) { return StringUtils.format("%s%s", child, chartCommand.toString()); } + private boolean isTimechartNode(Chart node) { + // A Chart node represents a timechart if it has a rowSplit that's an alias containing + // a span on the implicit timestamp field + if (node.getRowSplit() instanceof Alias) { + Alias alias = (Alias) node.getRowSplit(); + if (alias.getDelegated() instanceof Span) { + Span span = (Span) alias.getDelegated(); + return span.getField() instanceof Field + && "@timestamp".equals(((Field) span.getField()).getField().toString()); + } + } + return false; + } + public String visitRex(Rex node, String context) { String child = node.getChild().get(0).accept(this, context); String field = visitExpression(node.getField()); diff --git a/ppl/src/test/java/org/opensearch/sql/ppl/calcite/CalcitePPLTimechartTest.java b/ppl/src/test/java/org/opensearch/sql/ppl/calcite/CalcitePPLTimechartTest.java index c3ed1ebceea..33a9b84d23b 100644 --- a/ppl/src/test/java/org/opensearch/sql/ppl/calcite/CalcitePPLTimechartTest.java +++ b/ppl/src/test/java/org/opensearch/sql/ppl/calcite/CalcitePPLTimechartTest.java @@ -75,8 +75,9 @@ public void testTimechartBasic() { RelNode root = getRelNode(ppl); String expectedSparkSql = - "SELECT `SPAN`(`@timestamp`, 1, 'm') `@timestamp`, COUNT(*) `count`\n" + "SELECT `SPAN`(`@timestamp`, 1, 'm') `@timestamp`, COUNT(*) `count()`\n" + "FROM `scott`.`events`\n" + + "WHERE `@timestamp` IS NOT NULL\n" + "GROUP BY `SPAN`(`@timestamp`, 1, 'm')\n" + "ORDER BY 1 NULLS LAST"; verifyPPLToSparkSQL(root, expectedSparkSql); @@ -92,8 +93,9 @@ public void testTimechartPerSecond() { + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'm') `@timestamp`, SUM(`cpu_usage`)" + " `per_second(cpu_usage)`\n" + "FROM `scott`.`events`\n" + + "WHERE `@timestamp` IS NOT NULL AND `cpu_usage` IS NOT NULL\n" + "GROUP BY `SPAN`(`@timestamp`, 1, 'm')\n" - + "ORDER BY 1 NULLS LAST) `t2`"); + + "ORDER BY 1 NULLS LAST) `t3`"); } @Test @@ -106,8 +108,9 @@ public void testTimechartPerMinute() { + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'm') `@timestamp`, SUM(`cpu_usage`)" + " `per_minute(cpu_usage)`\n" + "FROM `scott`.`events`\n" + + "WHERE `@timestamp` IS NOT NULL AND `cpu_usage` IS NOT NULL\n" + "GROUP BY `SPAN`(`@timestamp`, 1, 'm')\n" - + "ORDER BY 1 NULLS LAST) `t2`"); + + "ORDER BY 1 NULLS LAST) `t3`"); } @Test @@ -120,8 +123,9 @@ public void testTimechartPerHour() { + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'm') `@timestamp`, SUM(`cpu_usage`)" + " `per_hour(cpu_usage)`\n" + "FROM `scott`.`events`\n" + + "WHERE `@timestamp` IS NOT NULL AND `cpu_usage` IS NOT NULL\n" + "GROUP BY `SPAN`(`@timestamp`, 1, 'm')\n" - + "ORDER BY 1 NULLS LAST) `t2`"); + + "ORDER BY 1 NULLS LAST) `t3`"); } @Test @@ -134,8 +138,9 @@ public void testTimechartPerDay() { + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'm') `@timestamp`, SUM(`cpu_usage`)" + " `per_day(cpu_usage)`\n" + "FROM `scott`.`events`\n" + + "WHERE `@timestamp` IS NOT NULL AND `cpu_usage` IS NOT NULL\n" + "GROUP BY `SPAN`(`@timestamp`, 1, 'm')\n" - + "ORDER BY 1 NULLS LAST) `t2`"); + + "ORDER BY 1 NULLS LAST) `t3`"); } @Test @@ -144,8 +149,9 @@ public void testTimechartWithSpan() { RelNode root = getRelNode(ppl); String expectedSparkSql = - "SELECT `SPAN`(`@timestamp`, 1, 'h') `@timestamp`, COUNT(*) `count`\n" + "SELECT `SPAN`(`@timestamp`, 1, 'h') `@timestamp`, COUNT(*) `count()`\n" + "FROM `scott`.`events`\n" + + "WHERE `@timestamp` IS NOT NULL\n" + "GROUP BY `SPAN`(`@timestamp`, 1, 'h')\n" + "ORDER BY 1 NULLS LAST"; verifyPPLToSparkSQL(root, expectedSparkSql); @@ -157,49 +163,24 @@ public void testTimechartWithLimit() { RelNode root = getRelNode(ppl); String expectedSparkSql = - "SELECT `@timestamp`, `host`, SUM(`actual_count`) `count`\n" - + "FROM (SELECT CAST(`t1`.`@timestamp` AS TIMESTAMP) `@timestamp`, CASE WHEN" - + " `t7`.`host` IS NOT NULL THEN `t1`.`host` ELSE CASE WHEN `t1`.`host` IS NULL THEN" - + " NULL ELSE 'OTHER' END END `host`, SUM(`t1`.`$f2_0`) `actual_count`\n" - + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'm') `@timestamp`, `host`, COUNT(*) `$f2_0`\n" + "SELECT `t2`.`@timestamp`, CASE WHEN `t2`.`host` IS NULL THEN 'NULL' WHEN" + + " `t9`.`_row_number_chart_` <= 3 THEN `t2`.`host` ELSE 'OTHER' END `host`," + + " SUM(`t2`.`count()`) `count()`\n" + + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'm') `@timestamp`, `host`, COUNT(*) `count()`\n" + "FROM `scott`.`events`\n" - + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'm')) `t1`\n" - + "LEFT JOIN (SELECT `host`, SUM(`$f2_0`) `grand_total`\n" - + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'm') `@timestamp`, `host`, COUNT(*) `$f2_0`\n" + + "WHERE `@timestamp` IS NOT NULL\n" + + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'm')) `t2`\n" + + "LEFT JOIN (SELECT `host`, SUM(`count()`) `__grand_total__`, ROW_NUMBER() OVER (ORDER" + + " BY SUM(`count()`) DESC) `_row_number_chart_`\n" + + "FROM (SELECT `host`, COUNT(*) `count()`\n" + "FROM `scott`.`events`\n" - + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'm')) `t4`\n" + + "WHERE `@timestamp` IS NOT NULL\n" + + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'm')) `t6`\n" + "WHERE `host` IS NOT NULL\n" - + "GROUP BY `host`\n" - + "ORDER BY 2 DESC NULLS FIRST\n" - + "LIMIT 3) `t7` ON `t1`.`host` IS NOT DISTINCT FROM `t7`.`host`\n" - + "GROUP BY CAST(`t1`.`@timestamp` AS TIMESTAMP), CASE WHEN `t7`.`host` IS NOT NULL" - + " THEN `t1`.`host` ELSE CASE WHEN `t1`.`host` IS NULL THEN NULL ELSE 'OTHER' END" - + " END\n" - + "UNION\n" - + "SELECT CAST(`t13`.`@timestamp` AS TIMESTAMP) `@timestamp`, `t24`.`$f0` `host`, 0" - + " `count`\n" - + "FROM (SELECT `@timestamp`\n" - + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'm') `@timestamp`\n" - + "FROM `scott`.`events`\n" - + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'm')) `t12`\n" - + "GROUP BY `@timestamp`) `t13`\n" - + "CROSS JOIN (SELECT CASE WHEN `t22`.`host` IS NOT NULL THEN `t16`.`host` ELSE CASE" - + " WHEN `t16`.`host` IS NULL THEN NULL ELSE 'OTHER' END END `$f0`\n" - + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'm') `@timestamp`, `host`, COUNT(*) `$f2_0`\n" - + "FROM `scott`.`events`\n" - + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'm')) `t16`\n" - + "LEFT JOIN (SELECT `host`, SUM(`$f2_0`) `grand_total`\n" - + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'm') `@timestamp`, `host`, COUNT(*) `$f2_0`\n" - + "FROM `scott`.`events`\n" - + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'm')) `t19`\n" - + "WHERE `host` IS NOT NULL\n" - + "GROUP BY `host`\n" - + "ORDER BY 2 DESC NULLS FIRST\n" - + "LIMIT 3) `t22` ON `t16`.`host` IS NOT DISTINCT FROM `t22`.`host`\n" - + "GROUP BY CASE WHEN `t22`.`host` IS NOT NULL THEN `t16`.`host` ELSE CASE WHEN" - + " `t16`.`host` IS NULL THEN NULL ELSE 'OTHER' END END) `t24`) `t26`\n" - + "GROUP BY `@timestamp`, `host`\n" - + "ORDER BY `@timestamp` NULLS LAST, `host` NULLS LAST"; + + "GROUP BY `host`) `t9` ON `t2`.`host` = `t9`.`host`\n" + + "GROUP BY `t2`.`@timestamp`, CASE WHEN `t2`.`host` IS NULL THEN 'NULL' WHEN" + + " `t9`.`_row_number_chart_` <= 3 THEN `t2`.`host` ELSE 'OTHER' END\n" + + "ORDER BY `t2`.`@timestamp` NULLS LAST, 2 NULLS LAST"; verifyPPLToSparkSQL(root, expectedSparkSql); } @@ -208,49 +189,24 @@ public void testTimechartWithSpan1h() { String ppl = "source=events | timechart span=1h count() by host"; RelNode root = getRelNode(ppl); String expectedSparkSql = - "SELECT `@timestamp`, `host`, SUM(`actual_count`) `count`\n" - + "FROM (SELECT CAST(`t1`.`@timestamp` AS TIMESTAMP) `@timestamp`, CASE WHEN" - + " `t7`.`host` IS NOT NULL THEN `t1`.`host` ELSE CASE WHEN `t1`.`host` IS NULL THEN" - + " NULL ELSE 'OTHER' END END `host`, SUM(`t1`.`$f2_0`) `actual_count`\n" - + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'h') `@timestamp`, `host`, COUNT(*) `$f2_0`\n" - + "FROM `scott`.`events`\n" - + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'h')) `t1`\n" - + "LEFT JOIN (SELECT `host`, SUM(`$f2_0`) `grand_total`\n" - + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'h') `@timestamp`, `host`, COUNT(*) `$f2_0`\n" - + "FROM `scott`.`events`\n" - + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'h')) `t4`\n" - + "WHERE `host` IS NOT NULL\n" - + "GROUP BY `host`\n" - + "ORDER BY 2 DESC NULLS FIRST\n" - + "LIMIT 10) `t7` ON `t1`.`host` IS NOT DISTINCT FROM `t7`.`host`\n" - + "GROUP BY CAST(`t1`.`@timestamp` AS TIMESTAMP), CASE WHEN `t7`.`host` IS NOT NULL" - + " THEN `t1`.`host` ELSE CASE WHEN `t1`.`host` IS NULL THEN NULL ELSE 'OTHER' END" - + " END\n" - + "UNION\n" - + "SELECT CAST(`t13`.`@timestamp` AS TIMESTAMP) `@timestamp`, `t24`.`$f0` `host`, 0" - + " `count`\n" - + "FROM (SELECT `@timestamp`\n" - + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'h') `@timestamp`\n" + "SELECT `t2`.`@timestamp`, CASE WHEN `t2`.`host` IS NULL THEN 'NULL' WHEN" + + " `t9`.`_row_number_chart_` <= 10 THEN `t2`.`host` ELSE 'OTHER' END `host`," + + " SUM(`t2`.`count()`) `count()`\n" + + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'h') `@timestamp`, `host`, COUNT(*) `count()`\n" + "FROM `scott`.`events`\n" - + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'h')) `t12`\n" - + "GROUP BY `@timestamp`) `t13`\n" - + "CROSS JOIN (SELECT CASE WHEN `t22`.`host` IS NOT NULL THEN `t16`.`host` ELSE CASE" - + " WHEN `t16`.`host` IS NULL THEN NULL ELSE 'OTHER' END END `$f0`\n" - + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'h') `@timestamp`, `host`, COUNT(*) `$f2_0`\n" + + "WHERE `@timestamp` IS NOT NULL\n" + + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'h')) `t2`\n" + + "LEFT JOIN (SELECT `host`, SUM(`count()`) `__grand_total__`, ROW_NUMBER() OVER (ORDER" + + " BY SUM(`count()`) DESC) `_row_number_chart_`\n" + + "FROM (SELECT `host`, COUNT(*) `count()`\n" + "FROM `scott`.`events`\n" - + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'h')) `t16`\n" - + "LEFT JOIN (SELECT `host`, SUM(`$f2_0`) `grand_total`\n" - + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'h') `@timestamp`, `host`, COUNT(*) `$f2_0`\n" - + "FROM `scott`.`events`\n" - + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'h')) `t19`\n" + + "WHERE `@timestamp` IS NOT NULL\n" + + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'h')) `t6`\n" + "WHERE `host` IS NOT NULL\n" - + "GROUP BY `host`\n" - + "ORDER BY 2 DESC NULLS FIRST\n" - + "LIMIT 10) `t22` ON `t16`.`host` IS NOT DISTINCT FROM `t22`.`host`\n" - + "GROUP BY CASE WHEN `t22`.`host` IS NOT NULL THEN `t16`.`host` ELSE CASE WHEN" - + " `t16`.`host` IS NULL THEN NULL ELSE 'OTHER' END END) `t24`) `t26`\n" - + "GROUP BY `@timestamp`, `host`\n" - + "ORDER BY `@timestamp` NULLS LAST, `host` NULLS LAST"; + + "GROUP BY `host`) `t9` ON `t2`.`host` = `t9`.`host`\n" + + "GROUP BY `t2`.`@timestamp`, CASE WHEN `t2`.`host` IS NULL THEN 'NULL' WHEN" + + " `t9`.`_row_number_chart_` <= 10 THEN `t2`.`host` ELSE 'OTHER' END\n" + + "ORDER BY `t2`.`@timestamp` NULLS LAST, 2 NULLS LAST"; verifyPPLToSparkSQL(root, expectedSparkSql); } @@ -259,25 +215,25 @@ public void testTimechartWithSpan1m() { String ppl = "source=events | timechart span=1m avg(cpu_usage) by region"; RelNode root = getRelNode(ppl); String expectedSparkSql = - "SELECT `t1`.`@timestamp`, CASE WHEN `t7`.`region` IS NOT NULL THEN `t1`.`region` ELSE CASE" - + " WHEN `t1`.`region` IS NULL THEN NULL ELSE 'OTHER' END END `region`, AVG(`t1`.`$f2`)" - + " `avg(cpu_usage)`\n" + "SELECT `t2`.`@timestamp`, CASE WHEN `t2`.`region` IS NULL THEN 'NULL' WHEN" + + " `t9`.`_row_number_chart_` <= 10 THEN `t2`.`region` ELSE 'OTHER' END `region`," + + " AVG(`t2`.`avg(cpu_usage)`) `avg(cpu_usage)`\n" + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'm') `@timestamp`, `region`, AVG(`cpu_usage`)" - + " `$f2`\n" + + " `avg(cpu_usage)`\n" + "FROM `scott`.`events`\n" - + "GROUP BY `region`, `SPAN`(`@timestamp`, 1, 'm')) `t1`\n" - + "LEFT JOIN (SELECT `region`, AVG(`$f2`) `grand_total`\n" - + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'm') `@timestamp`, `region`, AVG(`cpu_usage`)" - + " `$f2`\n" + + "WHERE `@timestamp` IS NOT NULL AND `cpu_usage` IS NOT NULL\n" + + "GROUP BY `region`, `SPAN`(`@timestamp`, 1, 'm')) `t2`\n" + + "LEFT JOIN (SELECT `region`, SUM(`avg(cpu_usage)`) `__grand_total__`, ROW_NUMBER()" + + " OVER (ORDER BY SUM(`avg(cpu_usage)`) DESC) `_row_number_chart_`\n" + + "FROM (SELECT `region`, AVG(`cpu_usage`) `avg(cpu_usage)`\n" + "FROM `scott`.`events`\n" - + "GROUP BY `region`, `SPAN`(`@timestamp`, 1, 'm')) `t4`\n" + + "WHERE `@timestamp` IS NOT NULL AND `cpu_usage` IS NOT NULL\n" + + "GROUP BY `region`, `SPAN`(`@timestamp`, 1, 'm')) `t6`\n" + "WHERE `region` IS NOT NULL\n" - + "GROUP BY `region`\n" - + "ORDER BY 2 DESC NULLS FIRST\n" - + "LIMIT 10) `t7` ON `t1`.`region` = `t7`.`region`\n" - + "GROUP BY `t1`.`@timestamp`, CASE WHEN `t7`.`region` IS NOT NULL THEN `t1`.`region`" - + " ELSE CASE WHEN `t1`.`region` IS NULL THEN NULL ELSE 'OTHER' END END\n" - + "ORDER BY `t1`.`@timestamp` NULLS LAST, 2 NULLS LAST"; + + "GROUP BY `region`) `t9` ON `t2`.`region` = `t9`.`region`\n" + + "GROUP BY `t2`.`@timestamp`, CASE WHEN `t2`.`region` IS NULL THEN 'NULL' WHEN" + + " `t9`.`_row_number_chart_` <= 10 THEN `t2`.`region` ELSE 'OTHER' END\n" + + "ORDER BY `t2`.`@timestamp` NULLS LAST, 2 NULLS LAST"; verifyPPLToSparkSQL(root, expectedSparkSql); } @@ -296,27 +252,26 @@ public void testTimechartWithLimitAndUseOtherFalse() { RelNode root = getRelNode(ppl); String expectedSparkSql = - "SELECT `t1`.`@timestamp`, CASE WHEN `t7`.`host` IS NOT NULL THEN `t1`.`host` ELSE CASE" - + " WHEN `t1`.`host` IS NULL THEN NULL ELSE 'OTHER' END END `host`, AVG(`t1`.`$f2`)" - + " `avg(cpu_usage)`\n" + "SELECT `t2`.`@timestamp`, CASE WHEN `t2`.`host` IS NULL THEN 'NULL' WHEN" + + " `t9`.`_row_number_chart_` <= 3 THEN `t2`.`host` ELSE 'OTHER' END `host`," + + " AVG(`t2`.`avg(cpu_usage)`) `avg(cpu_usage)`\n" + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'h') `@timestamp`, `host`, AVG(`cpu_usage`)" - + " `$f2`\n" + + " `avg(cpu_usage)`\n" + "FROM `scott`.`events`\n" - + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'h')) `t1`\n" - + "LEFT JOIN (SELECT `host`, AVG(`$f2`) `grand_total`\n" - + "FROM (SELECT `SPAN`(`@timestamp`, 1, 'h') `@timestamp`, `host`, AVG(`cpu_usage`)" - + " `$f2`\n" + + "WHERE `@timestamp` IS NOT NULL AND `cpu_usage` IS NOT NULL\n" + + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'h')) `t2`\n" + + "LEFT JOIN (SELECT `host`, SUM(`avg(cpu_usage)`) `__grand_total__`, ROW_NUMBER() OVER" + + " (ORDER BY SUM(`avg(cpu_usage)`) DESC) `_row_number_chart_`\n" + + "FROM (SELECT `host`, AVG(`cpu_usage`) `avg(cpu_usage)`\n" + "FROM `scott`.`events`\n" - + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'h')) `t4`\n" + + "WHERE `@timestamp` IS NOT NULL AND `cpu_usage` IS NOT NULL\n" + + "GROUP BY `host`, `SPAN`(`@timestamp`, 1, 'h')) `t6`\n" + "WHERE `host` IS NOT NULL\n" - + "GROUP BY `host`\n" - + "ORDER BY 2 DESC NULLS FIRST\n" - + "LIMIT 3) `t7` ON `t1`.`host` = `t7`.`host`\n" - + "GROUP BY `t1`.`@timestamp`, CASE WHEN `t7`.`host` IS NOT NULL THEN `t1`.`host` ELSE" - + " CASE WHEN `t1`.`host` IS NULL THEN NULL ELSE 'OTHER' END END\n" - + "HAVING CASE WHEN `t7`.`host` IS NOT NULL THEN `t1`.`host` ELSE CASE WHEN `t1`.`host`" - + " IS NULL THEN NULL ELSE 'OTHER' END END <> 'OTHER'\n" - + "ORDER BY `t1`.`@timestamp` NULLS LAST, 2 NULLS LAST"; + + "GROUP BY `host`) `t9` ON `t2`.`host` = `t9`.`host`\n" + + "WHERE `t9`.`_row_number_chart_` <= 3\n" + + "GROUP BY `t2`.`@timestamp`, CASE WHEN `t2`.`host` IS NULL THEN 'NULL' WHEN" + + " `t9`.`_row_number_chart_` <= 3 THEN `t2`.`host` ELSE 'OTHER' END\n" + + "ORDER BY `t2`.`@timestamp` NULLS LAST, 2 NULLS LAST"; verifyPPLToSparkSQL(root, expectedSparkSql); } diff --git a/ppl/src/test/java/org/opensearch/sql/ppl/parser/AstBuilderTest.java b/ppl/src/test/java/org/opensearch/sql/ppl/parser/AstBuilderTest.java index f1464e31065..94aac588511 100644 --- a/ppl/src/test/java/org/opensearch/sql/ppl/parser/AstBuilderTest.java +++ b/ppl/src/test/java/org/opensearch/sql/ppl/parser/AstBuilderTest.java @@ -79,7 +79,6 @@ import org.opensearch.sql.ast.tree.Kmeans; import org.opensearch.sql.ast.tree.ML; import org.opensearch.sql.ast.tree.RareTopN.CommandType; -import org.opensearch.sql.ast.tree.Timechart; import org.opensearch.sql.common.antlr.SyntaxCheckException; import org.opensearch.sql.common.setting.Settings; import org.opensearch.sql.common.setting.Settings.Key; @@ -1229,10 +1228,17 @@ public void testTimechartWithPerSecondFunction() { assertEqual( "source=t | timechart per_second(a)", eval( - new Timechart(relation("t"), alias("per_second(a)", aggregate("sum", field("a")))) - .span(span(field("@timestamp"), intLiteral(1), SpanUnit.of("m"))) - .limit(10) - .useOther(true), + Chart.builder() + .child(relation("t")) + .rowSplit( + alias("@timestamp", span(field("@timestamp"), intLiteral(1), SpanUnit.of("m")))) + .columnSplit(null) + .aggregationFunction(alias("per_second(a)", aggregate("sum", field("a")))) + .arguments( + exprList( + argument("limit", intLiteral(10)), + argument("useother", booleanLiteral(true)))) + .build(), let( field("per_second(a)"), function( @@ -1254,10 +1260,17 @@ public void testTimechartWithPerMinuteFunction() { assertEqual( "source=t | timechart per_minute(a)", eval( - new Timechart(relation("t"), alias("per_minute(a)", aggregate("sum", field("a")))) - .span(span(field("@timestamp"), intLiteral(1), SpanUnit.of("m"))) - .limit(10) - .useOther(true), + Chart.builder() + .child(relation("t")) + .rowSplit( + alias("@timestamp", span(field("@timestamp"), intLiteral(1), SpanUnit.of("m")))) + .columnSplit(null) + .aggregationFunction(alias("per_minute(a)", aggregate("sum", field("a")))) + .arguments( + exprList( + argument("limit", intLiteral(10)), + argument("useother", booleanLiteral(true)))) + .build(), let( field("per_minute(a)"), function( @@ -1279,10 +1292,17 @@ public void testTimechartWithPerHourFunction() { assertEqual( "source=t | timechart per_hour(a)", eval( - new Timechart(relation("t"), alias("per_hour(a)", aggregate("sum", field("a")))) - .span(span(field("@timestamp"), intLiteral(1), SpanUnit.of("m"))) - .limit(10) - .useOther(true), + Chart.builder() + .child(relation("t")) + .rowSplit( + alias("@timestamp", span(field("@timestamp"), intLiteral(1), SpanUnit.of("m")))) + .columnSplit(null) + .aggregationFunction(alias("per_hour(a)", aggregate("sum", field("a")))) + .arguments( + exprList( + argument("limit", intLiteral(10)), + argument("useother", booleanLiteral(true)))) + .build(), let( field("per_hour(a)"), function( @@ -1304,10 +1324,17 @@ public void testTimechartWithPerDayFunction() { assertEqual( "source=t | timechart per_day(a)", eval( - new Timechart(relation("t"), alias("per_day(a)", aggregate("sum", field("a")))) - .span(span(field("@timestamp"), intLiteral(1), SpanUnit.of("m"))) - .limit(10) - .useOther(true), + Chart.builder() + .child(relation("t")) + .rowSplit( + alias("@timestamp", span(field("@timestamp"), intLiteral(1), SpanUnit.of("m")))) + .columnSplit(null) + .aggregationFunction(alias("per_day(a)", aggregate("sum", field("a")))) + .arguments( + exprList( + argument("limit", intLiteral(10)), + argument("useother", booleanLiteral(true)))) + .build(), let( field("per_day(a)"), function( diff --git a/ppl/src/test/java/org/opensearch/sql/ppl/parser/AstExpressionBuilderTest.java b/ppl/src/test/java/org/opensearch/sql/ppl/parser/AstExpressionBuilderTest.java index 5d10960ea6b..c5accc7f7ca 100644 --- a/ppl/src/test/java/org/opensearch/sql/ppl/parser/AstExpressionBuilderTest.java +++ b/ppl/src/test/java/org/opensearch/sql/ppl/parser/AstExpressionBuilderTest.java @@ -63,7 +63,7 @@ import org.opensearch.sql.ast.expression.DataType; import org.opensearch.sql.ast.expression.RelevanceFieldList; import org.opensearch.sql.ast.expression.SpanUnit; -import org.opensearch.sql.ast.tree.Timechart; +import org.opensearch.sql.ast.tree.Chart; import org.opensearch.sql.calcite.plan.OpenSearchConstants; import org.opensearch.sql.common.antlr.SyntaxCheckException; @@ -1395,16 +1395,19 @@ public void testTimeModifierEarliestWithStringValue() { public void testTimechartSpanParameter() { assertEqual( "source=t | timechart span=30m count()", - Timechart.builder() + Chart.builder() .child(relation("t")) - .binExpression( - span( - field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), - intLiteral(30), - SpanUnit.m)) - .aggregateFunction(aggregate("count", allFields())) - .limit(10) - .useOther(true) + .rowSplit( + alias( + "@timestamp", + span( + field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), + intLiteral(30), + SpanUnit.m))) + .aggregationFunction(alias("count()", aggregate("count", allFields()))) + .arguments( + exprList( + argument("limit", intLiteral(10)), argument("useother", booleanLiteral(true)))) .build()); } @@ -1412,14 +1415,19 @@ public void testTimechartSpanParameter() { public void testTimechartLimitParameter() { assertEqual( "source=t | timechart limit=100 count()", - Timechart.builder() + Chart.builder() .child(relation("t")) - .binExpression( - span( - field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), intLiteral(1), SpanUnit.m)) - .aggregateFunction(aggregate("count", allFields())) - .limit(100) - .useOther(true) + .rowSplit( + alias( + "@timestamp", + span( + field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), + intLiteral(1), + SpanUnit.m))) + .aggregationFunction(alias("count()", aggregate("count", allFields()))) + .arguments( + exprList( + argument("limit", intLiteral(100)), argument("useother", booleanLiteral(true)))) .build()); } @@ -1434,26 +1442,36 @@ public void testTimechartNegativeLimitParameter() { public void testTimechartUseOtherWithBooleanLiteral() { assertEqual( "source=t | timechart useother=true count()", - Timechart.builder() + Chart.builder() .child(relation("t")) - .binExpression( - span( - field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), intLiteral(1), SpanUnit.m)) - .aggregateFunction(aggregate("count", allFields())) - .limit(10) - .useOther(true) + .rowSplit( + alias( + "@timestamp", + span( + field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), + intLiteral(1), + SpanUnit.m))) + .aggregationFunction(alias("count()", aggregate("count", allFields()))) + .arguments( + exprList( + argument("limit", intLiteral(10)), argument("useother", booleanLiteral(true)))) .build()); assertEqual( "source=t | timechart useother=false count()", - Timechart.builder() + Chart.builder() .child(relation("t")) - .binExpression( - span( - field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), intLiteral(1), SpanUnit.m)) - .aggregateFunction(aggregate("count", allFields())) - .limit(10) - .useOther(false) + .rowSplit( + alias( + "@timestamp", + span( + field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), + intLiteral(1), + SpanUnit.m))) + .aggregationFunction(alias("count()", aggregate("count", allFields()))) + .arguments( + exprList( + argument("limit", intLiteral(10)), argument("useother", booleanLiteral(false)))) .build()); } @@ -1461,50 +1479,70 @@ public void testTimechartUseOtherWithBooleanLiteral() { public void testTimechartUseOtherWithIdentifier() { assertEqual( "source=t | timechart useother=t count()", - Timechart.builder() + Chart.builder() .child(relation("t")) - .binExpression( - span( - field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), intLiteral(1), SpanUnit.m)) - .aggregateFunction(aggregate("count", allFields())) - .limit(10) - .useOther(true) + .rowSplit( + alias( + "@timestamp", + span( + field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), + intLiteral(1), + SpanUnit.m))) + .aggregationFunction(alias("count()", aggregate("count", allFields()))) + .arguments( + exprList( + argument("limit", intLiteral(10)), argument("useother", booleanLiteral(true)))) .build()); assertEqual( "source=t | timechart useother=f count()", - Timechart.builder() + Chart.builder() .child(relation("t")) - .binExpression( - span( - field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), intLiteral(1), SpanUnit.m)) - .aggregateFunction(aggregate("count", allFields())) - .limit(10) - .useOther(false) + .rowSplit( + alias( + "@timestamp", + span( + field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), + intLiteral(1), + SpanUnit.m))) + .aggregationFunction(alias("count()", aggregate("count", allFields()))) + .arguments( + exprList( + argument("limit", intLiteral(10)), argument("useother", booleanLiteral(false)))) .build()); assertEqual( "source=t | timechart useother=TRUE count()", - Timechart.builder() + Chart.builder() .child(relation("t")) - .binExpression( - span( - field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), intLiteral(1), SpanUnit.m)) - .aggregateFunction(aggregate("count", allFields())) - .limit(10) - .useOther(true) + .rowSplit( + alias( + "@timestamp", + span( + field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), + intLiteral(1), + SpanUnit.m))) + .aggregationFunction(alias("count()", aggregate("count", allFields()))) + .arguments( + exprList( + argument("limit", intLiteral(10)), argument("useother", booleanLiteral(true)))) .build()); assertEqual( "source=t | timechart useother=FALSE count()", - Timechart.builder() + Chart.builder() .child(relation("t")) - .binExpression( - span( - field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), intLiteral(1), SpanUnit.m)) - .aggregateFunction(aggregate("count", allFields())) - .limit(10) - .useOther(false) + .rowSplit( + alias( + "@timestamp", + span( + field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), + intLiteral(1), + SpanUnit.m))) + .aggregationFunction(alias("count()", aggregate("count", allFields()))) + .arguments( + exprList( + argument("limit", intLiteral(10)), argument("useother", booleanLiteral(false)))) .build()); } @@ -1568,42 +1606,55 @@ public void testVisitSpanLiteral() { // Test span literal with integer value and hour unit assertEqual( "source=t | timechart span=1h count()", - Timechart.builder() + Chart.builder() .child(relation("t")) - .binExpression( - span( - field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), intLiteral(1), SpanUnit.H)) - .aggregateFunction(aggregate("count", allFields())) - .limit(10) - .useOther(true) + .rowSplit( + alias( + "@timestamp", + span( + field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), + intLiteral(1), + SpanUnit.H))) + .aggregationFunction(alias("count()", aggregate("count", allFields()))) + .arguments( + exprList( + argument("limit", intLiteral(10)), argument("useother", booleanLiteral(true)))) .build()); // Test span literal with decimal value and minute unit assertEqual( "source=t | timechart span=2m count()", - Timechart.builder() + Chart.builder() .child(relation("t")) - .binExpression( - span( - field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), intLiteral(2), SpanUnit.m)) - .aggregateFunction(aggregate("count", allFields())) - .limit(10) - .useOther(true) + .rowSplit( + alias( + "@timestamp", + span( + field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), + intLiteral(2), + SpanUnit.m))) + .aggregationFunction(alias("count()", aggregate("count", allFields()))) + .arguments( + exprList( + argument("limit", intLiteral(10)), argument("useother", booleanLiteral(true)))) .build()); // Test span literal without unit (should use NONE unit) assertEqual( "source=t | timechart span=10 count()", - Timechart.builder() + Chart.builder() .child(relation("t")) - .binExpression( - span( - field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), - intLiteral(10), - SpanUnit.NONE)) - .aggregateFunction(aggregate("count", allFields())) - .limit(10) - .useOther(true) + .rowSplit( + alias( + "@timestamp", + span( + field(OpenSearchConstants.IMPLICIT_FIELD_TIMESTAMP), + intLiteral(10), + SpanUnit.NONE))) + .aggregationFunction(alias("count()", aggregate("count", allFields()))) + .arguments( + exprList( + argument("limit", intLiteral(10)), argument("useother", booleanLiteral(true)))) .build()); } 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 ec87000b5bf..8655775e269 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 @@ -255,7 +255,7 @@ public void testReverseCommand() { @Test public void testTimechartCommand() { assertEquals( - "source=table | timechart span=span(identifier, *** m) limit=10 useother=true count() by" + "source=table | timechart limit=*** useother=*** count() by span(identifier, *** m)" + " identifier", anonymize("source=t | timechart count() by host")); }