Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Translate PPL dedup Command Part 2: allowedDuplication>1 #543

Merged
merged 9 commits into from
Aug 30, 2024
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -187,7 +187,7 @@ class FlintSparkPPLDedupITSuite
assert(ex.getMessage.contains("Consecutive deduplication is not supported"))
}

ignore("test dedupe 2 name") {
test("test dedupe 2 name") {
val frame = sql(s"""
| source = $testTable| dedup 2 name | fields name
| """.stripMargin)
Expand All @@ -200,7 +200,7 @@ class FlintSparkPPLDedupITSuite
assert(results.sorted.sameElements(expectedResults.sorted))
}

ignore("test dedupe 2 name, category") {
test("test dedupe 2 name, category") {
val frame = sql(s"""
| source = $testTable| dedup 2 name, category | fields name, category
| """.stripMargin)
Expand All @@ -225,7 +225,7 @@ class FlintSparkPPLDedupITSuite
assert(results.sorted.sameElements(expectedResults.sorted))
}

ignore("test dedupe 2 name KEEPEMPTY=true") {
test("test dedupe 2 name KEEPEMPTY=true") {
val frame = sql(s"""
| source = $testTable| dedup 2 name KEEPEMPTY=true | fields name, category
| """.stripMargin)
Expand Down Expand Up @@ -259,7 +259,7 @@ class FlintSparkPPLDedupITSuite
.sameElements(expectedResults.sorted.map(_.getAs[String](0))))
}

ignore("test dedupe 2 name, category KEEPEMPTY=true") {
test("test dedupe 2 name, category KEEPEMPTY=true") {
val frame = sql(s"""
| source = $testTable| dedup 2 name, category KEEPEMPTY=true | fields name, category
| """.stripMargin)
Expand Down
8 changes: 5 additions & 3 deletions ppl-spark-integration/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -282,9 +282,11 @@ Limitation: Overriding existing field is unsupported, following queries throw ex
- `source = table | dedup 1 a,b | fields a,b,c`
- `source = table | dedup 1 a keepempty=true | fields a,b,c`
- `source = table | dedup 1 a,b keepempty=true | fields a,b,c`
- `source = table | dedup 1 a consecutive=true| fields a,b,c` (Unsupported)
- `source = table | dedup 2 a | fields a,b,c` (Unsupported)

- `source = table | dedup 2 a | fields a,b,c`
- `source = table | dedup 2 a,b | fields a,b,c`
- `source = table | dedup 2 a keepempty=true | fields a,b,c`
- `source = table | dedup 2 a,b keepempty=true | fields a,b,c`
- `source = table | dedup 1 a consecutive=true| fields a,b,c` (Consecutive deduplication is unsupported)

For additional details on PPL commands - view [PPL Commands Docs](https://github.com/opensearch-project/sql/blob/main/docs/user/ppl/index.rst)

Expand Down
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi
visitDedup(...) is too large (>136 lines) and needs to be extracted to a dedicated DedupUtils class or similar
we should try to reduce CatalystQueryPlanVisitor only to be the command pattern consolidator and move the actual business code away into dedicated strategy (commands)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

Original file line number Diff line number Diff line change
Expand Up @@ -10,10 +10,12 @@
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation;
import org.apache.spark.sql.catalyst.analysis.UnresolvedStar$;
import org.apache.spark.sql.catalyst.expressions.Expression;
import org.apache.spark.sql.catalyst.expressions.LessThanOrEqual;
import org.apache.spark.sql.catalyst.expressions.NamedExpression;
import org.apache.spark.sql.catalyst.expressions.Predicate;
import org.apache.spark.sql.catalyst.expressions.SortOrder;
import org.apache.spark.sql.catalyst.plans.logical.Aggregate;
import org.apache.spark.sql.catalyst.plans.logical.DataFrameDropColumns;
import org.apache.spark.sql.catalyst.plans.logical.Deduplicate;
import org.apache.spark.sql.catalyst.plans.logical.DescribeRelation$;
import org.apache.spark.sql.catalyst.plans.logical.Limit;
Expand Down Expand Up @@ -64,6 +66,7 @@
import org.opensearch.sql.ppl.utils.BuiltinFunctionTranslator;
import org.opensearch.sql.ppl.utils.ComparatorTransformer;
import org.opensearch.sql.ppl.utils.SortUtils;
import org.opensearch.sql.ppl.utils.WindowSpecTransformer;
import scala.Option;
import scala.Option$;
import scala.collection.Seq;
Expand Down Expand Up @@ -318,13 +321,14 @@ public LogicalPlan visitDedupe(Dedupe node, CatalystPlanContext context) {
// adding Aggregate operator could achieve better performance.
if (allowedDuplication == 1) {
if (keepEmpty) {
// | dedup a, b keepempty=true
// Union
// :- Deduplicate ['a, 'b]
// : +- Filter (isnotnull('a) AND isnotnull('b)
// : +- Project
// : +- ...
// : +- UnresolvedRelation
// +- Filter (isnull('a) OR isnull('a))
// +- Project
// +- ...
// +- UnresolvedRelation

context.apply(p -> {
Expand All @@ -339,9 +343,10 @@ public LogicalPlan visitDedupe(Dedupe node, CatalystPlanContext context) {
});
return context.getPlan();
} else {
// | dedup a, b keepempty=false
// Deduplicate ['a, 'b]
// +- Filter (isnotnull('a) AND isnotnull('b))
// +- Project
// +- ...
// +- UnresolvedRelation

Expression isNotNullExpr = buildIsNotNullFilterExpression(node, context);
Expand All @@ -350,8 +355,86 @@ public LogicalPlan visitDedupe(Dedupe node, CatalystPlanContext context) {
return context.apply(p -> new Deduplicate(dedupFields, p));
}
} else {
// TODO
throw new UnsupportedOperationException("Number of duplicate events greater than 1 is not supported");
if (keepEmpty) {
// | dedup 2 a, b keepempty=true
// Union
//:- DataFrameDropColumns('_row_number_)
//: +- Filter ('_row_number_ <= 2)
//: +- Window [row_number() windowspecdefinition('a, 'b, 'a ASC NULLS FIRST, 'b ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_], ['a, 'b], ['a ASC NULLS FIRST, 'b ASC NULLS FIRST]
//: +- Filter (isnotnull('a) AND isnotnull('b))
//: +- ...
//: +- UnresolvedRelation
//+- Filter (isnull('a) OR isnull('b))
// +- ...
// +- UnresolvedRelation

context.apply(p -> {
// Build isnull Filter for right
Expression isNullExpr = buildIsNullFilterExpression(node, context);
LogicalPlan right = new org.apache.spark.sql.catalyst.plans.logical.Filter(isNullExpr, p);

// Build isnotnull Filter
Expression isNotNullExpr = buildIsNotNullFilterExpression(node, context);
LogicalPlan isNotNullFilter = new org.apache.spark.sql.catalyst.plans.logical.Filter(isNotNullExpr, p);

// Build Window
visitFieldList(node.getFields(), context);
Seq<Expression> partitionSpec = context.retainAllNamedParseExpressions(exp -> exp);
visitFieldList(node.getFields(), context);
Seq<SortOrder> orderSpec = context.retainAllNamedParseExpressions(exp -> SortUtils.sortOrder(exp, true));
NamedExpression rowNumber = WindowSpecTransformer.buildRowNumber(partitionSpec, orderSpec);
LogicalPlan window = new org.apache.spark.sql.catalyst.plans.logical.Window(
seq(rowNumber),
partitionSpec,
orderSpec,
isNotNullFilter);

// Build deduplication Filter ('_row_number_ <= n)
Expression filterExpr = new LessThanOrEqual(
rowNumber.toAttribute(),
new org.apache.spark.sql.catalyst.expressions.Literal(allowedDuplication, DataTypes.IntegerType));
LogicalPlan deduplicationFilter = new org.apache.spark.sql.catalyst.plans.logical.Filter(filterExpr, window);

// Build DataFrameDropColumns('_row_number_) for left
LogicalPlan left = new DataFrameDropColumns(seq(rowNumber.toAttribute()), deduplicationFilter);

// Build Union
return new Union(seq(left, right), false, false);
});
return context.getPlan();
} else {
// | dedup 2 a, b keepempty=false
// DataFrameDropColumns('row_number_col)
// +- Filter ('_row_number_ <= n)
// +- Window [row_number() windowspecdefinition('a, 'b, 'a ASC NULLS FIRST, 'b ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_], ['a, 'b], ['a ASC NULLS FIRST, 'b ASC NULLS FIRST]
// +- Filter (isnotnull('a) AND isnotnull('b))
// +- ...
// +- UnresolvedRelation

// Build isnotnull Filter
Expression isNotNullExpr = buildIsNotNullFilterExpression(node, context);
context.apply(p -> new org.apache.spark.sql.catalyst.plans.logical.Filter(isNotNullExpr, p));

// Build Window
visitFieldList(node.getFields(), context);
Seq<Expression> partitionSpec = context.retainAllNamedParseExpressions(exp -> exp);
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

does the order of the dedup fields keeped? for instance, dedup 2, b, a, will the output be a, b?

Copy link
Member Author

@LantaoJin LantaoJin Aug 27, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes. the original order will be kept. I added several tests in IT to verify this case.

visitFieldList(node.getFields(), context);
Seq<SortOrder> orderSpec = context.retainAllNamedParseExpressions(exp -> SortUtils.sortOrder(exp, true));
NamedExpression rowNumber = WindowSpecTransformer.buildRowNumber(partitionSpec, orderSpec);
context.apply(p -> new org.apache.spark.sql.catalyst.plans.logical.Window(
seq(rowNumber),
partitionSpec,
orderSpec, p));

// Build deduplication Filter ('_row_number_ <= n)
Expression filterExpr = new LessThanOrEqual(
rowNumber.toAttribute(),
new org.apache.spark.sql.catalyst.expressions.Literal(allowedDuplication, DataTypes.IntegerType));
context.apply(p -> new org.apache.spark.sql.catalyst.plans.logical.Filter(filterExpr, p));

// Build DataFrameDropColumns('_row_number_) Spark 3.5.1+ required
return context.apply(p -> new DataFrameDropColumns(seq(rowNumber.toAttribute()), p));
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,24 +5,37 @@

package org.opensearch.sql.ppl.utils;

import org.apache.spark.sql.catalyst.expressions.CurrentRow$;
import org.apache.spark.sql.catalyst.expressions.Divide;
import org.apache.spark.sql.catalyst.expressions.Expression;
import org.apache.spark.sql.catalyst.expressions.Floor;
import org.apache.spark.sql.catalyst.expressions.Literal;
import org.apache.spark.sql.catalyst.expressions.Multiply;
import org.apache.spark.sql.catalyst.expressions.NamedExpression;
import org.apache.spark.sql.catalyst.expressions.RowFrame$;
import org.apache.spark.sql.catalyst.expressions.RowNumber;
import org.apache.spark.sql.catalyst.expressions.SortOrder;
import org.apache.spark.sql.catalyst.expressions.SpecifiedWindowFrame;
import org.apache.spark.sql.catalyst.expressions.TimeWindow;
import org.apache.spark.sql.types.DateType$;
import org.apache.spark.sql.types.StringType$;
import org.apache.spark.sql.catalyst.expressions.UnboundedPreceding$;
import org.apache.spark.sql.catalyst.expressions.WindowExpression;
import org.apache.spark.sql.catalyst.expressions.WindowSpecDefinition;
import org.opensearch.sql.ast.expression.SpanUnit;
import scala.Option;
import scala.collection.Seq;

import java.util.ArrayList;

import static java.lang.String.format;
import static org.opensearch.sql.ast.expression.DataType.STRING;
import static org.opensearch.sql.ast.expression.SpanUnit.NONE;
import static org.opensearch.sql.ast.expression.SpanUnit.UNKNOWN;
import static org.opensearch.sql.ppl.utils.DataTypeTransformer.seq;
import static org.opensearch.sql.ppl.utils.DataTypeTransformer.translate;

public interface WindowSpecTransformer {

String ROW_NUMBER_COLUMN_NAME = "_row_number_";

/**
* create a static window buckets based on the given value
*
Expand Down Expand Up @@ -50,4 +63,20 @@ static org.apache.spark.sql.catalyst.expressions.Literal timeLiteral( Expression
return new org.apache.spark.sql.catalyst.expressions.Literal(
translate(format, STRING), translate(STRING));
}

static NamedExpression buildRowNumber(Seq<Expression> partitionSpec, Seq<SortOrder> orderSpec) {
WindowExpression rowNumber = new WindowExpression(
new RowNumber(),
new WindowSpecDefinition(
partitionSpec,
orderSpec,
new SpecifiedWindowFrame(RowFrame$.MODULE$, UnboundedPreceding$.MODULE$, CurrentRow$.MODULE$)));
return org.apache.spark.sql.catalyst.expressions.Alias$.MODULE$.apply(
rowNumber,
ROW_NUMBER_COLUMN_NAME,
NamedExpression.newExprId(),
seq(new ArrayList<String>()),
Option.empty(),
seq(new ArrayList<String>()));
}
}
Loading
Loading