-
Notifications
You must be signed in to change notification settings - Fork 33
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
Support CAST function #952
Changes from 3 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,119 @@ | ||
/* | ||
* Copyright OpenSearch Contributors | ||
* SPDX-License-Identifier: Apache-2.0 | ||
*/ | ||
|
||
package org.opensearch.flint.spark.ppl | ||
|
||
import java.sql.Date | ||
import java.sql.Timestamp | ||
|
||
import org.apache.spark.sql.{QueryTest, Row} | ||
import org.apache.spark.sql.streaming.StreamTest | ||
|
||
class FlintSparkPPLCastITSuite | ||
extends QueryTest | ||
with LogicalPlanTestUtils | ||
with FlintPPLSuite | ||
with StreamTest { | ||
|
||
/** Test table and index name */ | ||
private val testTable = "spark_catalog.default.flint_ppl_test" | ||
|
||
override def beforeAll(): Unit = { | ||
super.beforeAll() | ||
// Create test table | ||
createNullableJsonContentTable(testTable) | ||
} | ||
|
||
protected override def afterEach(): Unit = { | ||
super.afterEach() | ||
// Stop all streaming jobs if any | ||
spark.streams.active.foreach { job => | ||
job.stop() | ||
job.awaitTermination() | ||
} | ||
} | ||
|
||
test("test cast number to compatible data types") { | ||
val frame = sql(s""" | ||
| source=$testTable | eval | ||
| id_string = cast(id as string), | ||
| id_double = cast(id as double), | ||
| id_long = cast(id as long), | ||
| id_boolean = cast(id as boolean) | ||
| | fields id, id_string, id_double, id_long, id_boolean | head 1 | ||
| """.stripMargin) | ||
|
||
assert( | ||
frame.dtypes.sameElements( | ||
Array( | ||
("id", "IntegerType"), | ||
("id_string", "StringType"), | ||
("id_double", "DoubleType"), | ||
("id_long", "LongType"), | ||
("id_boolean", "BooleanType")))) | ||
assertSameRows(Seq(Row(1, "1", 1.0, 1L, true)), frame) | ||
} | ||
|
||
test("test cast string to compatible data types") { | ||
val frame = sql(s""" | ||
| source=$testTable | eval | ||
| id_int = cast(cast(id as string) as int), | ||
| cast_true = cast("True" as boolean), | ||
| cast_false = cast("false" as boolean), | ||
| cast_timestamp = cast("2024-11-26 23:39:06" as timestamp), | ||
| cast_date = cast("2024-11-26" as date) | ||
| | fields id_int, cast_true, cast_false, cast_timestamp, cast_date | head 1 | ||
| """.stripMargin) | ||
|
||
assert( | ||
frame.dtypes.sameElements( | ||
Array( | ||
("id_int", "IntegerType"), | ||
("cast_true", "BooleanType"), | ||
("cast_false", "BooleanType"), | ||
("cast_timestamp", "TimestampType"), | ||
("cast_date", "DateType")))) | ||
assertSameRows( | ||
Seq( | ||
Row( | ||
1, | ||
true, | ||
false, | ||
Timestamp.valueOf("2024-11-26 23:39:06"), | ||
Date.valueOf("2024-11-26"))), | ||
frame) | ||
} | ||
|
||
test("test cast time related types to compatible data types") { | ||
val frame = sql(s""" | ||
| source=$testTable | eval | ||
| timestamp = cast("2024-11-26 23:39:06" as timestamp), | ||
| ts_str = cast(timestamp as string), | ||
| ts_date = cast(timestamp as date), | ||
| date_str = cast(ts_date as string), | ||
| date_ts = cast(ts_date as timestamp) | ||
| | fields timestamp, ts_str, ts_date, date_str, date_ts | head 1 | ||
| """.stripMargin) | ||
|
||
assert( | ||
frame.dtypes.sameElements( | ||
Array( | ||
("timestamp", "TimestampType"), | ||
("ts_str", "StringType"), | ||
("ts_date", "DateType"), | ||
("date_str", "StringType"), | ||
("date_ts", "TimestampType")))) | ||
assertSameRows( | ||
Seq( | ||
Row( | ||
Timestamp.valueOf("2024-11-26 23:39:06"), | ||
"2024-11-26 23:39:06", | ||
Date.valueOf("2024-11-26"), | ||
"2024-11-26", | ||
Timestamp.valueOf("2024-11-26 00:00:00"))), | ||
frame) | ||
} | ||
|
||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,39 @@ | ||
/* | ||
* Copyright OpenSearch Contributors | ||
* SPDX-License-Identifier: Apache-2.0 | ||
*/ | ||
|
||
package org.opensearch.sql.ast.expression; | ||
|
||
import java.util.Collections; | ||
import java.util.List; | ||
import lombok.EqualsAndHashCode; | ||
import lombok.Getter; | ||
import lombok.RequiredArgsConstructor; | ||
import org.opensearch.sql.ast.AbstractNodeVisitor; | ||
|
||
/** | ||
* Expression node of cast | ||
*/ | ||
@Getter | ||
@EqualsAndHashCode(callSuper = false) | ||
@RequiredArgsConstructor | ||
public class Cast extends UnresolvedExpression { | ||
private final UnresolvedExpression expression; | ||
private final DataType dataType; | ||
|
||
@Override | ||
public List<UnresolvedExpression> getChild() { | ||
return Collections.singletonList(expression); | ||
} | ||
|
||
@Override | ||
public <R, C> R accept(AbstractNodeVisitor<R, C> nodeVisitor, C context) { | ||
return nodeVisitor.visitCast(this, context); | ||
} | ||
|
||
@Override | ||
public String toString() { | ||
return String.format("CAST(%s AS %s)", expression, dataType); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,6 +19,7 @@ | |
import org.opensearch.sql.ast.expression.AttributeList; | ||
import org.opensearch.sql.ast.expression.Between; | ||
import org.opensearch.sql.ast.expression.Case; | ||
import org.opensearch.sql.ast.expression.Cast; | ||
import org.opensearch.sql.ast.expression.Cidr; | ||
import org.opensearch.sql.ast.expression.Compare; | ||
import org.opensearch.sql.ast.expression.DataType; | ||
|
@@ -279,9 +280,9 @@ public UnresolvedExpression visitEvalFunctionCall(OpenSearchPPLParser.EvalFuncti | |
return buildFunction(ctx.evalFunctionName().getText(), ctx.functionArgs().functionArg()); | ||
} | ||
|
||
@Override | ||
public UnresolvedExpression visitConvertedDataType(OpenSearchPPLParser.ConvertedDataTypeContext ctx) { | ||
return new Literal(ctx.getText(), DataType.STRING); | ||
@Override public UnresolvedExpression visitDataTypeFunctionCall(OpenSearchPPLParser.DataTypeFunctionCallContext ctx) { | ||
// TODO: for long term consideration, needs to implement DataTypeBuilder/Visitor to parse all data types | ||
return new Cast(this.visit(ctx.expression()), DataType.fromString(ctx.convertedDataType().getText())); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @qianheng-aws can u plz check is it simpler using this DataTypeTransformer here instead ? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I used DataTypeTransformer in CatalystExpressionVisitor. DataTypeTransformer will translate opensearch datatype into spark datatype, while it's all about opensearch AST here. |
||
} | ||
|
||
@Override | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This line could be deleted as well.