You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by hv...@apache.org on 2023/02/15 18:43:09 UTC
[spark] branch master updated: [SPARK-42441][CONNECT] Scala Client add Column APIs
This is an automated email from the ASF dual-hosted git repository.
hvanhovell pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push:
new d4a5beb45b8 [SPARK-42441][CONNECT] Scala Client add Column APIs
d4a5beb45b8 is described below
commit d4a5beb45b8a6fbde3b7d252fcd83755cef97aa3
Author: Herman van Hovell <he...@databricks.com>
AuthorDate: Wed Feb 15 14:42:52 2023 -0400
[SPARK-42441][CONNECT] Scala Client add Column APIs
### What changes were proposed in this pull request?
This PR adds most Column APIs for the Spark Connect Scala Client.
### Why are the changes needed?
We want the Scala Client to have API parity with the existing SparkSession/Dataset APIs.
### How was this patch tested?
Golden files, and I added a test for local behavior.
Closes #40027 from hvanhovell/SPARK-42441.
Authored-by: Herman van Hovell <he...@databricks.com>
Signed-off-by: Herman van Hovell <he...@databricks.com>
---
.../main/scala/org/apache/spark/sql/Column.scala | 1071 +++++++++++++++++++-
.../main/scala/org/apache/spark/sql/Dataset.scala | 2 +-
.../scala/org/apache/spark/sql/functions.scala | 28 +
.../org/apache/spark/sql/ColumnTestSuite.scala | 180 ++++
.../apache/spark/sql/PlanGenerationTestSuite.scala | 240 ++++-
.../query-tests/explain-results/column_add.explain | 2 +-
.../explain-results/column_alias.explain | 2 +-
.../query-tests/explain-results/column_and.explain | 2 +
.../explain-results/column_apply.explain | 2 +
.../explain-results/column_as_multi.explain | 2 +
.../column_as_with_metadata.explain | 2 +
.../query-tests/explain-results/column_asc.explain | 2 +
.../explain-results/column_asc_nulls_first.explain | 2 +
.../explain-results/column_asc_nulls_last.explain | 2 +
.../explain-results/column_between.explain | 2 +
.../explain-results/column_bitwiseAND.explain | 2 +
.../explain-results/column_bitwiseOR.explain | 2 +
.../explain-results/column_bitwiseXOR.explain | 2 +
.../explain-results/column_by_name.explain | 2 -
.../explain-results/column_cast.explain | 2 +
.../explain-results/column_contains.explain | 2 +
.../explain-results/column_desc.explain | 2 +
.../column_desc_nulls_first.explain | 2 +
.../explain-results/column_desc_nulls_last.explain | 2 +
.../explain-results/column_divide.explain | 2 +
.../explain-results/column_dropFields.explain | 2 +
.../explain-results/column_endsWith.explain | 2 +
.../explain-results/column_eqNullSafe.explain | 2 +
.../explain-results/column_equals.explain | 2 +-
.../query-tests/explain-results/column_geq.explain | 2 +
.../explain-results/column_getField.explain | 2 +
.../explain-results/column_getItem.explain | 2 +
.../query-tests/explain-results/column_gt.explain | 2 +
.../explain-results/column_ilike.explain | 2 +
.../explain-results/column_isNaN.explain | 2 +
.../explain-results/column_isNotNull.explain | 2 +
.../explain-results/column_isNull.explain | 2 +
.../explain-results/column_isin.explain | 2 +
.../query-tests/explain-results/column_leq.explain | 2 +
.../explain-results/column_like.explain | 2 +
.../query-tests/explain-results/column_lt.explain | 2 +
.../explain-results/column_modulo.explain | 2 +
.../explain-results/column_multiply.explain | 2 +
.../query-tests/explain-results/column_not.explain | 2 +
.../explain-results/column_not_equals.explain | 2 +
.../query-tests/explain-results/column_or.explain | 2 +
.../explain-results/column_rlike.explain | 2 +
.../explain-results/column_star.explain | 2 +
.../column_star_with_target.explain | 2 +
.../explain-results/column_startsWith.explain | 2 +
.../explain-results/column_substr.explain | 2 +
.../explain-results/column_subtract.explain | 2 +
.../explain-results/column_unary_minus.explain | 2 +
.../explain-results/column_when_otherwise.explain | 2 +
.../explain-results/column_withField.explain | 2 +
.../explain-results/orderBy_columns.explain | 2 +-
.../explain-results/orderBy_strings.explain | 2 +-
.../explain-results/sort_columns.explain | 2 +-
.../explain-results/sort_strings.explain | 2 +-
.../resources/query-tests/queries/column_add.json | 2 +-
.../query-tests/queries/column_add.proto.bin | 4 +-
.../query-tests/queries/column_alias.json | 2 +-
.../query-tests/queries/column_alias.proto.bin | 4 +-
.../resources/query-tests/queries/column_and.json | 41 +
.../query-tests/queries/column_and.proto.bin | Bin 0 -> 204 bytes
.../query-tests/queries/column_apply.json | 23 +
.../query-tests/queries/column_apply.proto.bin | 5 +
.../query-tests/queries/column_as_multi.json | 19 +
.../query-tests/queries/column_as_multi.proto.bin | 4 +
.../queries/column_as_with_metadata.json | 20 +
.../queries/column_as_with_metadata.proto.bin | 4 +
.../queries/{sort_strings.json => column_asc.json} | 12 +-
.../query-tests/queries/column_asc.proto.bin | 4 +
...rt_strings.json => column_asc_nulls_first.json} | 12 +-
.../queries/column_asc_nulls_first.proto.bin | 4 +
.../query-tests/queries/column_asc_nulls_last.json | 19 +
.../queries/column_asc_nulls_last.proto.bin | 4 +
.../query-tests/queries/column_between.json | 41 +
.../query-tests/queries/column_between.proto.bin | 10 +
.../query-tests/queries/column_bitwiseAND.json | 23 +
.../queries/column_bitwiseAND.proto.bin | 5 +
.../{column_add.json => column_bitwiseOR.json} | 8 +-
.../query-tests/queries/column_bitwiseOR.proto.bin | 5 +
.../{column_add.json => column_bitwiseXOR.json} | 8 +-
.../queries/column_bitwiseXOR.proto.bin | 5 +
.../query-tests/queries/column_by_name.json | 14 -
.../query-tests/queries/column_by_name.proto.bin | 3 -
.../resources/query-tests/queries/column_cast.json | 22 +
.../query-tests/queries/column_cast.proto.bin | Bin 0 -> 163 bytes
.../query-tests/queries/column_contains.json | 23 +
.../query-tests/queries/column_contains.proto.bin | 5 +
.../resources/query-tests/queries/column_desc.json | 19 +
.../query-tests/queries/column_desc.proto.bin | 4 +
.../queries/column_desc_nulls_first.json | 19 +
.../queries/column_desc_nulls_first.proto.bin | 4 +
.../queries/column_desc_nulls_last.json | 19 +
.../queries/column_desc_nulls_last.proto.bin | 4 +
.../{column_add.json => column_divide.json} | 4 +-
.../query-tests/queries/column_divide.proto.bin | 5 +
.../query-tests/queries/column_dropFields.json | 24 +
.../queries/column_dropFields.proto.bin | 6 +
.../query-tests/queries/column_endsWith.json | 23 +
.../query-tests/queries/column_endsWith.proto.bin | 5 +
.../{column_add.json => column_eqNullSafe.json} | 4 +-
.../queries/column_eqNullSafe.proto.bin | 5 +
.../query-tests/queries/column_equals.json | 2 +-
.../query-tests/queries/column_equals.proto.bin | 4 +-
.../queries/{column_add.json => column_geq.json} | 4 +-
.../query-tests/queries/column_geq.proto.bin | 5 +
.../query-tests/queries/column_getField.json | 23 +
.../query-tests/queries/column_getField.proto.bin | 5 +
.../query-tests/queries/column_getItem.json | 23 +
.../query-tests/queries/column_getItem.proto.bin | 5 +
.../queries/{column_add.json => column_gt.json} | 4 +-
.../query-tests/queries/column_gt.proto.bin | 5 +
.../query-tests/queries/column_ilike.json | 23 +
.../query-tests/queries/column_ilike.proto.bin | 5 +
.../queries/{column_add.json => column_isNaN.json} | 8 +-
.../query-tests/queries/column_isNaN.proto.bin | 4 +
.../query-tests/queries/column_isNotNull.json | 19 +
.../query-tests/queries/column_isNotNull.proto.bin | 4 +
.../query-tests/queries/column_isNull.json | 19 +
.../query-tests/queries/column_isNull.proto.bin | 4 +
.../resources/query-tests/queries/column_isin.json | 31 +
.../query-tests/queries/column_isin.proto.bin | 7 +
.../queries/{column_add.json => column_leq.json} | 4 +-
.../query-tests/queries/column_leq.proto.bin | 5 +
.../resources/query-tests/queries/column_like.json | 23 +
.../query-tests/queries/column_like.proto.bin | 5 +
.../queries/{column_add.json => column_lt.json} | 4 +-
.../query-tests/queries/column_lt.proto.bin | 5 +
.../{column_add.json => column_modulo.json} | 8 +-
.../query-tests/queries/column_modulo.proto.bin | 5 +
.../{column_add.json => column_multiply.json} | 4 +-
.../query-tests/queries/column_multiply.proto.bin | 5 +
.../resources/query-tests/queries/column_not.json | 19 +
.../query-tests/queries/column_not.proto.bin | 4 +
.../query-tests/queries/column_not_equals.json | 28 +
.../queries/column_not_equals.proto.bin | 6 +
.../resources/query-tests/queries/column_or.json | 41 +
.../query-tests/queries/column_or.proto.bin | Bin 0 -> 203 bytes
.../query-tests/queries/column_rlike.json | 23 +
.../query-tests/queries/column_rlike.proto.bin | 6 +
.../resources/query-tests/queries/column_star.json | 13 +
.../query-tests/queries/column_star.proto.bin | Bin 0 -> 152 bytes
.../queries/column_star_with_target.json | 14 +
.../queries/column_star_with_target.proto.bin | 3 +
.../query-tests/queries/column_startsWith.json | 23 +
.../queries/column_startsWith.proto.bin | 6 +
.../query-tests/queries/column_substr.json | 27 +
.../query-tests/queries/column_substr.proto.bin | 6 +
.../{column_add.json => column_subtract.json} | 4 +-
.../query-tests/queries/column_subtract.proto.bin | 5 +
.../query-tests/queries/column_unary_minus.json | 19 +
.../queries/column_unary_minus.proto.bin | 4 +
.../query-tests/queries/column_when_otherwise.json | 53 +
.../queries/column_when_otherwise.proto.bin | 14 +
.../query-tests/queries/column_withField.json | 24 +
.../query-tests/queries/column_withField.proto.bin | 5 +
.../query-tests/queries/orderBy_columns.json | 2 +-
.../query-tests/queries/orderBy_columns.proto.bin | Bin 82 -> 82 bytes
.../query-tests/queries/orderBy_strings.json | 2 +-
.../query-tests/queries/orderBy_strings.proto.bin | Bin 82 -> 82 bytes
.../query-tests/queries/sort_columns.json | 2 +-
.../query-tests/queries/sort_columns.proto.bin | Bin 69 -> 69 bytes
.../query-tests/queries/sort_strings.json | 2 +-
.../query-tests/queries/sort_strings.proto.bin | Bin 68 -> 68 bytes
.../org/apache/spark/sql/connect/dsl/package.scala | 2 +-
168 files changed, 2622 insertions(+), 136 deletions(-)
diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Column.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Column.scala
index 1634c2d5b05..638029308d2 100644
--- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Column.scala
+++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Column.scala
@@ -22,9 +22,10 @@ import org.apache.spark.connect.proto
import org.apache.spark.connect.proto.Expression.SortOrder.NullOrdering
import org.apache.spark.connect.proto.Expression.SortOrder.SortDirection
import org.apache.spark.internal.Logging
-import org.apache.spark.sql.Column.fn
-import org.apache.spark.sql.connect.client.unsupported
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.connect.common.DataTypeProtoConverter
import org.apache.spark.sql.functions.lit
+import org.apache.spark.sql.types.{DataType, Metadata}
/**
* A column that will be computed based on the data in a `DataFrame`.
@@ -49,36 +50,974 @@ import org.apache.spark.sql.functions.lit
*/
class Column private[sql] (private[sql] val expr: proto.Expression) extends Logging {
+ private def fn(name: String): Column = Column.fn(name, this)
+ private def fn(name: String, other: Column): Column = Column.fn(name, this, other)
+ private def fn(name: String, other: Any): Column = Column.fn(name, this, lit(other))
+
+ override def toString: String = expr.toString
+
+ override def equals(that: Any): Boolean = that match {
+ case that: Column => expr == that.expr
+ case _ => false
+ }
+
+ override def hashCode: Int = expr.hashCode()
+
+ /**
+ * Extracts a value or values from a complex type. The following types of extraction are
+ * supported:
+ * - Given an Array, an integer ordinal can be used to retrieve a single value.
+ * - Given a Map, a key of the correct type can be used to retrieve an individual value.
+ * - Given a Struct, a string fieldName can be used to extract that field.
+ * - Given an Array of Structs, a string fieldName can be used to extract filed of every
+ * struct in that array, and return an Array of fields.
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def apply(extraction: Any): Column = Column { builder =>
+ builder.getUnresolvedExtractValueBuilder
+ .setChild(expr)
+ .setExtraction(lit(extraction).expr)
+ }
+
+ /**
+ * Unary minus, i.e. negate the expression.
+ * {{{
+ * // Scala: select the amount column and negates all values.
+ * df.select( -df("amount") )
+ *
+ * // Java:
+ * import static org.apache.spark.sql.functions.*;
+ * df.select( negate(col("amount") );
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def unary_- : Column = fn("negative")
+
+ /**
+ * Inversion of boolean expression, i.e. NOT.
+ * {{{
+ * // Scala: select rows that are not active (isActive === false)
+ * df.filter( !df("isActive") )
+ *
+ * // Java:
+ * import static org.apache.spark.sql.functions.*;
+ * df.filter( not(df.col("isActive")) );
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def unary_! : Column = fn("!")
+
+ /**
+ * Equality test.
+ * {{{
+ * // Scala:
+ * df.filter( df("colA") === df("colB") )
+ *
+ * // Java
+ * import static org.apache.spark.sql.functions.*;
+ * df.filter( col("colA").equalTo(col("colB")) );
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def ===(other: Any): Column = fn("=", other)
+
+ /**
+ * Equality test.
+ * {{{
+ * // Scala:
+ * df.filter( df("colA") === df("colB") )
+ *
+ * // Java
+ * import static org.apache.spark.sql.functions.*;
+ * df.filter( col("colA").equalTo(col("colB")) );
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def equalTo(other: Any): Column = this === other
+
+ /**
+ * Inequality test.
+ * {{{
+ * // Scala:
+ * df.select( df("colA") =!= df("colB") )
+ * df.select( !(df("colA") === df("colB")) )
+ *
+ * // Java:
+ * import static org.apache.spark.sql.functions.*;
+ * df.filter( col("colA").notEqual(col("colB")) );
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def =!=(other: Any): Column = !(this === other)
+
+ /**
+ * Inequality test.
+ * {{{
+ * // Scala:
+ * df.select( df("colA") !== df("colB") )
+ * df.select( !(df("colA") === df("colB")) )
+ *
+ * // Java:
+ * import static org.apache.spark.sql.functions.*;
+ * df.filter( col("colA").notEqual(col("colB")) );
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ @deprecated("!== does not have the same precedence as ===, use =!= instead", "2.0.0")
+ def !==(other: Any): Column = this =!= other
+
+ /**
+ * Inequality test.
+ * {{{
+ * // Scala:
+ * df.select( df("colA") !== df("colB") )
+ * df.select( !(df("colA") === df("colB")) )
+ *
+ * // Java:
+ * import static org.apache.spark.sql.functions.*;
+ * df.filter( col("colA").notEqual(col("colB")) );
+ * }}}
+ *
+ * @group java_expr_ops
+ * @since 3.4.0
+ */
+ def notEqual(other: Any): Column = this =!= other
+
+ /**
+ * Greater than.
+ * {{{
+ * // Scala: The following selects people older than 21.
+ * people.select( people("age") > 21 )
+ *
+ * // Java:
+ * import static org.apache.spark.sql.functions.*;
+ * people.select( people.col("age").gt(21) );
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def >(other: Any): Column = fn(">", other)
+
+ /**
+ * Greater than.
+ * {{{
+ * // Scala: The following selects people older than 21.
+ * people.select( people("age") > lit(21) )
+ *
+ * // Java:
+ * import static org.apache.spark.sql.functions.*;
+ * people.select( people.col("age").gt(21) );
+ * }}}
+ *
+ * @group java_expr_ops
+ * @since 3.4.0
+ */
+ def gt(other: Any): Column = this > other
+
+ /**
+ * Less than.
+ * {{{
+ * // Scala: The following selects people younger than 21.
+ * people.select( people("age") < 21 )
+ *
+ * // Java:
+ * people.select( people.col("age").lt(21) );
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def <(other: Any): Column = fn("<", other)
+
+ /**
+ * Less than.
+ * {{{
+ * // Scala: The following selects people younger than 21.
+ * people.select( people("age") < 21 )
+ *
+ * // Java:
+ * people.select( people.col("age").lt(21) );
+ * }}}
+ *
+ * @group java_expr_ops
+ * @since 3.4.0
+ */
+ def lt(other: Any): Column = this < other
+
+ /**
+ * Less than or equal to.
+ * {{{
+ * // Scala: The following selects people age 21 or younger than 21.
+ * people.select( people("age") <= 21 )
+ *
+ * // Java:
+ * people.select( people.col("age").leq(21) );
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def <=(other: Any): Column = fn("<=", other)
+
+ /**
+ * Less than or equal to.
+ * {{{
+ * // Scala: The following selects people age 21 or younger than 21.
+ * people.select( people("age") <= 21 )
+ *
+ * // Java:
+ * people.select( people.col("age").leq(21) );
+ * }}}
+ *
+ * @group java_expr_ops
+ * @since 3.4.0
+ */
+ def leq(other: Any): Column = this <= other
+
+ /**
+ * Greater than or equal to an expression.
+ * {{{
+ * // Scala: The following selects people age 21 or older than 21.
+ * people.select( people("age") >= 21 )
+ *
+ * // Java:
+ * people.select( people.col("age").geq(21) )
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def >=(other: Any): Column = fn(">=", other)
+
+ /**
+ * Greater than or equal to an expression.
+ * {{{
+ * // Scala: The following selects people age 21 or older than 21.
+ * people.select( people("age") >= 21 )
+ *
+ * // Java:
+ * people.select( people.col("age").geq(21) )
+ * }}}
+ *
+ * @group java_expr_ops
+ * @since 3.4.0
+ */
+ def geq(other: Any): Column = this >= other
+
+ /**
+ * Equality test that is safe for null values.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def <=>(other: Any): Column = fn("<=>", other)
+
+ /**
+ * Equality test that is safe for null values.
+ *
+ * @group java_expr_ops
+ * @since 3.4.0
+ */
+ def eqNullSafe(other: Any): Column = this <=> other
+
+ private def extractWhen(name: String): java.util.List[proto.Expression] = {
+ def fail(): Nothing = {
+ throw new IllegalArgumentException(
+ s"$name() can only be applied on a Column previously generated by when() function")
+ }
+ if (!expr.hasUnresolvedFunction) {
+ fail()
+ }
+ val parentFn = expr.getUnresolvedFunction
+ if (parentFn.getFunctionName != "when") {
+ fail()
+ }
+ parentFn.getArgumentsList
+ }
+
+ /**
+ * Evaluates a list of conditions and returns one of multiple possible result expressions. If
+ * otherwise is not defined at the end, null is returned for unmatched conditions.
+ *
+ * {{{
+ * // Example: encoding gender string column into integer.
+ *
+ * // Scala:
+ * people.select(when(people("gender") === "male", 0)
+ * .when(people("gender") === "female", 1)
+ * .otherwise(2))
+ *
+ * // Java:
+ * people.select(when(col("gender").equalTo("male"), 0)
+ * .when(col("gender").equalTo("female"), 1)
+ * .otherwise(2))
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def when(condition: Column, value: Any): Column = {
+ val expressions = extractWhen("when")
+ if (expressions.size() % 2 == 1) {
+ throw new IllegalArgumentException("when() cannot be applied once otherwise() is applied")
+ }
+ Column { builder =>
+ builder.getUnresolvedFunctionBuilder
+ .setFunctionName("when")
+ .addAllArguments(expressions)
+ .addArguments(condition.expr)
+ .addArguments(lit(value).expr)
+ }
+ }
+
+ /**
+ * Evaluates a list of conditions and returns one of multiple possible result expressions. If
+ * otherwise is not defined at the end, null is returned for unmatched conditions.
+ *
+ * {{{
+ * // Example: encoding gender string column into integer.
+ *
+ * // Scala:
+ * people.select(when(people("gender") === "male", 0)
+ * .when(people("gender") === "female", 1)
+ * .otherwise(2))
+ *
+ * // Java:
+ * people.select(when(col("gender").equalTo("male"), 0)
+ * .when(col("gender").equalTo("female"), 1)
+ * .otherwise(2))
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def otherwise(value: Any): Column = {
+ val expressions = extractWhen("otherwise")
+ if (expressions.size() % 2 == 1) {
+ throw new IllegalArgumentException(
+ "otherwise() can only be applied once on a Column previously generated by when()")
+ }
+ Column { builder =>
+ builder.getUnresolvedFunctionBuilder
+ .setFunctionName("when")
+ .addAllArguments(expressions)
+ .addArguments(lit(value).expr)
+ }
+ }
+
+ /**
+ * True if the current column is between the lower bound and upper bound, inclusive.
+ *
+ * @group java_expr_ops
+ * @since 3.4.0
+ */
+ def between(lowerBound: Any, upperBound: Any): Column = {
+ (this >= lowerBound) && (this <= upperBound)
+ }
+
+ /**
+ * True if the current expression is NaN.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def isNaN: Column = fn("isNaN")
+
+ /**
+ * True if the current expression is null.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def isNull: Column = fn("isNull")
+
+ /**
+ * True if the current expression is NOT null.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def isNotNull: Column = fn("isNotNull")
+
+ /**
+ * Boolean OR.
+ * {{{
+ * // Scala: The following selects people that are in school or employed.
+ * people.filter( people("inSchool") || people("isEmployed") )
+ *
+ * // Java:
+ * people.filter( people.col("inSchool").or(people.col("isEmployed")) );
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def ||(other: Any): Column = fn("or", other)
+
+ /**
+ * Boolean OR.
+ * {{{
+ * // Scala: The following selects people that are in school or employed.
+ * people.filter( people("inSchool") || people("isEmployed") )
+ *
+ * // Java:
+ * people.filter( people.col("inSchool").or(people.col("isEmployed")) );
+ * }}}
+ *
+ * @group java_expr_ops
+ * @since 3.4.0
+ */
+ def or(other: Column): Column = this || other
+
+ /**
+ * Boolean AND.
+ * {{{
+ * // Scala: The following selects people that are in school and employed at the same time.
+ * people.select( people("inSchool") && people("isEmployed") )
+ *
+ * // Java:
+ * people.select( people.col("inSchool").and(people.col("isEmployed")) );
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def &&(other: Any): Column = fn("and", other)
+
+ /**
+ * Boolean AND.
+ * {{{
+ * // Scala: The following selects people that are in school and employed at the same time.
+ * people.select( people("inSchool") && people("isEmployed") )
+ *
+ * // Java:
+ * people.select( people.col("inSchool").and(people.col("isEmployed")) );
+ * }}}
+ *
+ * @group java_expr_ops
+ * @since 3.4.0
+ */
+ def and(other: Column): Column = this && other
+
+ /**
+ * Sum of this expression and another expression.
+ * {{{
+ * // Scala: The following selects the sum of a person's height and weight.
+ * people.select( people("height") + people("weight") )
+ *
+ * // Java:
+ * people.select( people.col("height").plus(people.col("weight")) );
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def +(other: Any): Column = fn("+", other)
+
+ /**
+ * Sum of this expression and another expression.
+ * {{{
+ * // Scala: The following selects the sum of a person's height and weight.
+ * people.select( people("height") + people("weight") )
+ *
+ * // Java:
+ * people.select( people.col("height").plus(people.col("weight")) );
+ * }}}
+ *
+ * @group java_expr_ops
+ * @since 3.4.0
+ */
+ def plus(other: Any): Column = this + other
+
+ /**
+ * Subtraction. Subtract the other expression from this expression.
+ * {{{
+ * // Scala: The following selects the difference between people's height and their weight.
+ * people.select( people("height") - people("weight") )
+ *
+ * // Java:
+ * people.select( people.col("height").minus(people.col("weight")) );
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def -(other: Any): Column = fn("-", other)
+
+ /**
+ * Subtraction. Subtract the other expression from this expression.
+ * {{{
+ * // Scala: The following selects the difference between people's height and their weight.
+ * people.select( people("height") - people("weight") )
+ *
+ * // Java:
+ * people.select( people.col("height").minus(people.col("weight")) );
+ * }}}
+ *
+ * @group java_expr_ops
+ * @since 3.4.0
+ */
+ def minus(other: Any): Column = this - other
+
+ /**
+ * Multiplication of this expression and another expression.
+ * {{{
+ * // Scala: The following multiplies a person's height by their weight.
+ * people.select( people("height") * people("weight") )
+ *
+ * // Java:
+ * people.select( people.col("height").multiply(people.col("weight")) );
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def *(other: Any): Column = fn("*", other)
+
+ /**
+ * Multiplication of this expression and another expression.
+ * {{{
+ * // Scala: The following multiplies a person's height by their weight.
+ * people.select( people("height") * people("weight") )
+ *
+ * // Java:
+ * people.select( people.col("height").multiply(people.col("weight")) );
+ * }}}
+ *
+ * @group java_expr_ops
+ * @since 3.4.0
+ */
+ def multiply(other: Any): Column = this * other
+
+ /**
+ * Division this expression by another expression.
+ * {{{
+ * // Scala: The following divides a person's height by their weight.
+ * people.select( people("height") / people("weight") )
+ *
+ * // Java:
+ * people.select( people.col("height").divide(people.col("weight")) );
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def /(other: Any): Column = fn("/", other)
+
+ /**
+ * Division this expression by another expression.
+ * {{{
+ * // Scala: The following divides a person's height by their weight.
+ * people.select( people("height") / people("weight") )
+ *
+ * // Java:
+ * people.select( people.col("height").divide(people.col("weight")) );
+ * }}}
+ *
+ * @group java_expr_ops
+ * @since 3.4.0
+ */
+ def divide(other: Any): Column = this / other
+
+ /**
+ * Modulo (a.k.a. remainder) expression.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def %(other: Any): Column = fn("%", other)
+
+ /**
+ * Modulo (a.k.a. remainder) expression.
+ *
+ * @group java_expr_ops
+ * @since 3.4.0
+ */
+ def mod(other: Any): Column = this % other
+
/**
- * Sum of this expression and another expression.
+ * A boolean expression that is evaluated to true if the value of this expression is contained
+ * by the evaluated values of the arguments.
+ *
+ * Note: Since the type of the elements in the list are inferred only during the run time, the
+ * elements will be "up-casted" to the most common type for comparison. For eg: 1) In the case
+ * of "Int vs String", the "Int" will be up-casted to "String" and the comparison will look like
+ * "String vs String". 2) In the case of "Float vs Double", the "Float" will be up-casted to
+ * "Double" and the comparison will look like "Double vs Double"
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ @scala.annotation.varargs
+ def isin(list: Any*): Column = Column.fn("in", this +: list.map(lit): _*)
+
+ /**
+ * A boolean expression that is evaluated to true if the value of this expression is contained
+ * by the provided collection.
+ *
+ * Note: Since the type of the elements in the collection are inferred only during the run time,
+ * the elements will be "up-casted" to the most common type for comparison. For eg: 1) In the
+ * case of "Int vs String", the "Int" will be up-casted to "String" and the comparison will look
+ * like "String vs String". 2) In the case of "Float vs Double", the "Float" will be up-casted
+ * to "Double" and the comparison will look like "Double vs Double"
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def isInCollection(values: scala.collection.Iterable[_]): Column = isin(values.toSeq: _*)
+
+ /**
+ * A boolean expression that is evaluated to true if the value of this expression is contained
+ * by the provided collection.
+ *
+ * Note: Since the type of the elements in the collection are inferred only during the run time,
+ * the elements will be "up-casted" to the most common type for comparison. For eg: 1) In the
+ * case of "Int vs String", the "Int" will be up-casted to "String" and the comparison will look
+ * like "String vs String". 2) In the case of "Float vs Double", the "Float" will be up-casted
+ * to "Double" and the comparison will look like "Double vs Double"
+ *
+ * @group java_expr_ops
+ * @since 3.4.0
+ */
+ def isInCollection(values: java.lang.Iterable[_]): Column = isInCollection(values.asScala)
+
+ /**
+ * SQL like expression. Returns a boolean column based on a SQL LIKE match.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def like(literal: String): Column = fn("like", literal)
+
+ /**
+ * SQL RLIKE expression (LIKE with Regex). Returns a boolean column based on a regex match.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def rlike(literal: String): Column = fn("rlike", literal)
+
+ /**
+ * SQL ILIKE expression (case insensitive LIKE).
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def ilike(literal: String): Column = fn("ilike", literal)
+
+ /**
+ * An expression that gets an item at position `ordinal` out of an array, or gets a value by key
+ * `key` in a `MapType`.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def getItem(key: Any): Column = apply(key)
+
+ // scalastyle:off line.size.limit
+ /**
+ * An expression that adds/replaces field in `StructType` by name.
+ *
* {{{
- * // Scala: The following selects the sum of a person's height and weight.
- * people.select( people("height") + people("weight") )
+ * val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+ * df.select($"struct_col".withField("c", lit(3)))
+ * // result: {"a":1,"b":2,"c":3}
*
- * // Java:
- * people.select( people.col("height").plus(people.col("weight")) );
+ * val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+ * df.select($"struct_col".withField("b", lit(3)))
+ * // result: {"a":1,"b":3}
+ *
+ * val df = sql("SELECT CAST(NULL AS struct<a:int,b:int>) struct_col")
+ * df.select($"struct_col".withField("c", lit(3)))
+ * // result: null of type struct<a:int,b:int,c:int>
+ *
+ * val df = sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col")
+ * df.select($"struct_col".withField("b", lit(100)))
+ * // result: {"a":1,"b":100,"b":100}
+ *
+ * val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+ * df.select($"struct_col".withField("a.c", lit(3)))
+ * // result: {"a":{"a":1,"b":2,"c":3}}
+ *
+ * val df = sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col")
+ * df.select($"struct_col".withField("a.c", lit(3)))
+ * // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
+ * }}}
+ *
+ * This method supports adding/replacing nested fields directly e.g.
+ *
+ * {{{
+ * val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+ * df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4)))
+ * // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+ * }}}
+ *
+ * However, if you are going to add/replace multiple nested fields, it is more optimal to
+ * extract out the nested struct before adding/replacing multiple fields e.g.
+ *
+ * {{{
+ * val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+ * df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4))))
+ * // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
* }}}
*
* @group expr_ops
* @since 3.4.0
*/
- def +(other: Any): Column = fn("+", this, lit(other))
+ // scalastyle:on line.size.limit
+ def withField(fieldName: String, col: Column): Column = {
+ require(fieldName != null, "fieldName cannot be null")
+ require(col != null, "col cannot be null")
+ Column { builder =>
+ builder.getUpdateFieldsBuilder
+ .setStructExpression(expr)
+ .setFieldName(fieldName)
+ .setValueExpression(col.expr)
+ }
+ }
+ // scalastyle:off line.size.limit
/**
- * Equality test.
+ * An expression that drops fields in `StructType` by name. This is a no-op if schema doesn't
+ * contain field name(s).
+ *
* {{{
- * // Scala:
- * df.filter( df("colA") === df("colB") )
+ * val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+ * df.select($"struct_col".dropFields("b"))
+ * // result: {"a":1}
*
- * // Java
- * import static org.apache.spark.sql.functions.*;
- * df.filter( col("colA").equalTo(col("colB")) );
+ * val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+ * df.select($"struct_col".dropFields("c"))
+ * // result: {"a":1,"b":2}
+ *
+ * val df = sql("SELECT named_struct('a', 1, 'b', 2, 'c', 3) struct_col")
+ * df.select($"struct_col".dropFields("b", "c"))
+ * // result: {"a":1}
+ *
+ * val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+ * df.select($"struct_col".dropFields("a", "b"))
+ * // result: org.apache.spark.sql.AnalysisException: [DATATYPE_MISMATCH.CANNOT_DROP_ALL_FIELDS] Cannot resolve "update_fields(struct_col, dropfield(), dropfield())" due to data type mismatch: Cannot drop all fields in struct.;
+ *
+ * val df = sql("SELECT CAST(NULL AS struct<a:int,b:int>) struct_col")
+ * df.select($"struct_col".dropFields("b"))
+ * // result: null of type struct<a:int>
+ *
+ * val df = sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col")
+ * df.select($"struct_col".dropFields("b"))
+ * // result: {"a":1}
+ *
+ * val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+ * df.select($"struct_col".dropFields("a.b"))
+ * // result: {"a":{"a":1}}
+ *
+ * val df = sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col")
+ * df.select($"struct_col".dropFields("a.c"))
+ * // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
+ * }}}
+ *
+ * This method supports dropping multiple nested fields directly e.g.
+ *
+ * {{{
+ * val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+ * df.select($"struct_col".dropFields("a.b", "a.c"))
+ * // result: {"a":{"a":1}}
+ * }}}
+ *
+ * However, if you are going to drop multiple nested fields, it is more optimal to extract out
+ * the nested struct before dropping multiple fields from it e.g.
+ *
+ * {{{
+ * val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+ * df.select($"struct_col".withField("a", $"struct_col.a".dropFields("b", "c")))
+ * // result: {"a":{"a":1}}
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ // scalastyle:on line.size.limit
+ def dropFields(fieldNames: String*): Column = {
+ fieldNames.foldLeft(this) { case (column, fieldName) =>
+ Column { builder =>
+ builder.getUpdateFieldsBuilder
+ .setStructExpression(column.expr)
+ .setFieldName(fieldName)
+ }
+ }
+ }
+
+ /**
+ * An expression that gets a field by name in a `StructType`.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def getField(fieldName: String): Column = apply(fieldName)
+
+ /**
+ * An expression that returns a substring.
+ * @param startPos
+ * expression for the starting position.
+ * @param len
+ * expression for the length of the substring.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def substr(startPos: Column, len: Column): Column = Column.fn("substr", this, startPos, len)
+
+ /**
+ * An expression that returns a substring.
+ * @param startPos
+ * starting position.
+ * @param len
+ * length of the substring.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def substr(startPos: Int, len: Int): Column = substr(lit(startPos), lit(len))
+
+ /**
+ * Contains the other element. Returns a boolean column based on a string match.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def contains(other: Any): Column = fn("contains", other)
+
+ /**
+ * String starts with. Returns a boolean column based on a string match.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def startsWith(other: Column): Column = fn("startswith", other)
+
+ /**
+ * String starts with another string literal. Returns a boolean column based on a string match.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def startsWith(literal: String): Column = startsWith(lit(literal))
+
+ /**
+ * String ends with. Returns a boolean column based on a string match.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def endsWith(other: Column): Column = fn("endswith", other)
+
+ /**
+ * String ends with another string literal. Returns a boolean column based on a string match.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def endsWith(literal: String): Column = endsWith(lit(literal))
+
+ /**
+ * Gives the column an alias. Same as `as`.
+ * {{{
+ * // Renames colA to colB in select output.
+ * df.select($"colA".alias("colB"))
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def alias(alias: String): Column = name(alias)
+
+ /**
+ * Gives the column an alias.
+ * {{{
+ * // Renames colA to colB in select output.
+ * df.select($"colA".as("colB"))
+ * }}}
+ *
+ * If the current column has metadata associated with it, this metadata will be propagated to
+ * the new column. If this not desired, use the API `as(alias: String, metadata: Metadata)` with
+ * explicit metadata.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def as(alias: String): Column = name(alias)
+
+ /**
+ * (Scala-specific) Assigns the given aliases to the results of a table generating function.
+ * {{{
+ * // Renames colA to colB in select output.
+ * df.select(explode($"myMap").as("key" :: "value" :: Nil))
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def as(aliases: Seq[String]): Column = Column { builder =>
+ builder.getAliasBuilder.setExpr(expr).addAllName(aliases.asJava)
+ }
+
+ /**
+ * Assigns the given aliases to the results of a table generating function.
+ * {{{
+ * // Renames colA to colB in select output.
+ * df.select(explode($"myMap").as("key" :: "value" :: Nil))
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def as(aliases: Array[String]): Column = as(aliases.toSeq)
+
+ /**
+ * Gives the column an alias.
+ * {{{
+ * // Renames colA to colB in select output.
+ * df.select($"colA".as("colB"))
+ * }}}
+ *
+ * If the current column has metadata associated with it, this metadata will be propagated to
+ * the new column. If this not desired, use the API `as(alias: String, metadata: Metadata)` with
+ * explicit metadata.
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def as(alias: Symbol): Column = name(alias.name)
+
+ /**
+ * Gives the column an alias with metadata.
+ * {{{
+ * val metadata: Metadata = ...
+ * df.select($"colA".as("colB", metadata))
* }}}
*
* @group expr_ops
* @since 3.4.0
*/
- def ===(other: Any): Column = fn("=", this, lit(other))
+ def as(alias: String, metadata: Metadata): Column = Column { builder =>
+ builder.getAliasBuilder
+ .setExpr(expr)
+ .addName(alias)
+ .setMetadata(metadata.json)
+ }
/**
* Gives the column a name (alias).
@@ -94,10 +1033,42 @@ class Column private[sql] (private[sql] val expr: proto.Expression) extends Logg
* @group expr_ops
* @since 3.4.0
*/
- def name(alias: String): Column = Column { builder =>
- builder.getAliasBuilder.addName(alias).setExpr(expr)
+ def name(alias: String): Column = as(alias :: Nil)
+
+ /**
+ * Casts the column to a different data type.
+ * {{{
+ * // Casts colA to IntegerType.
+ * import org.apache.spark.sql.types.IntegerType
+ * df.select(df("colA").cast(IntegerType))
+ *
+ * // equivalent to
+ * df.select(df("colA").cast("int"))
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def cast(to: DataType): Column = Column { builder =>
+ builder.getCastBuilder
+ .setExpr(expr)
+ .setType(DataTypeProtoConverter.toConnectProtoType(to))
}
+ /**
+ * Casts the column to a different data type, using the canonical string representation of the
+ * type. The supported types are: `string`, `boolean`, `byte`, `short`, `int`, `long`, `float`,
+ * `double`, `decimal`, `date`, `timestamp`.
+ * {{{
+ * // Casts colA to integer.
+ * df.select(df("colA").cast("int"))
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def cast(to: String): Column = cast(CatalystSqlParser.parseDataType(to))
+
/**
* Returns a sort expression based on the descending order of the column.
* {{{
@@ -109,7 +1080,7 @@ class Column private[sql] (private[sql] val expr: proto.Expression) extends Logg
* }}}
*
* @group expr_ops
- * @since 1.3.0
+ * @since 3.4.0
*/
def desc: Column = desc_nulls_last
@@ -125,7 +1096,7 @@ class Column private[sql] (private[sql] val expr: proto.Expression) extends Logg
* }}}
*
* @group expr_ops
- * @since 2.1.0
+ * @since 3.4.0
*/
def desc_nulls_first: Column =
buildSortOrder(SortDirection.SORT_DIRECTION_DESCENDING, NullOrdering.SORT_NULLS_FIRST)
@@ -142,7 +1113,7 @@ class Column private[sql] (private[sql] val expr: proto.Expression) extends Logg
* }}}
*
* @group expr_ops
- * @since 2.1.0
+ * @since 3.4.0
*/
def desc_nulls_last: Column =
buildSortOrder(SortDirection.SORT_DIRECTION_DESCENDING, NullOrdering.SORT_NULLS_LAST)
@@ -158,7 +1129,7 @@ class Column private[sql] (private[sql] val expr: proto.Expression) extends Logg
* }}}
*
* @group expr_ops
- * @since 1.3.0
+ * @since 3.4.0
*/
def asc: Column = asc_nulls_first
@@ -174,7 +1145,7 @@ class Column private[sql] (private[sql] val expr: proto.Expression) extends Logg
* }}}
*
* @group expr_ops
- * @since 2.1.0
+ * @since 3.4.0
*/
def asc_nulls_first: Column =
buildSortOrder(SortDirection.SORT_DIRECTION_ASCENDING, NullOrdering.SORT_NULLS_FIRST)
@@ -191,18 +1162,19 @@ class Column private[sql] (private[sql] val expr: proto.Expression) extends Logg
* }}}
*
* @group expr_ops
- * @since 2.1.0
+ * @since 3.4.0
*/
def asc_nulls_last: Column =
buildSortOrder(SortDirection.SORT_DIRECTION_ASCENDING, NullOrdering.SORT_NULLS_LAST)
- private def buildSortOrder(sortDirection: SortDirection, nullOrdering: NullOrdering): Column =
+ private def buildSortOrder(sortDirection: SortDirection, nullOrdering: NullOrdering): Column = {
Column { builder =>
builder.getSortOrderBuilder
.setChild(expr)
.setDirection(sortDirection)
.setNullOrdering(nullOrdering)
}
+ }
private[sql] def sortOrder: proto.Expression.SortOrder = {
val base = if (expr.hasSortOrder) {
@@ -212,6 +1184,55 @@ class Column private[sql] (private[sql] val expr: proto.Expression) extends Logg
}
base.getSortOrder
}
+
+ /**
+ * Prints the expression to the console for debugging purposes.
+ *
+ * @group df_ops
+ * @since 3.4.0
+ */
+ def explain(extended: Boolean): Unit = {
+ // scalastyle:off println
+ if (extended) {
+ println(expr)
+ } else {
+ println(toString)
+ }
+ // scalastyle:on println
+ }
+
+ /**
+ * Compute bitwise OR of this expression with another expression.
+ * {{{
+ * df.select($"colA".bitwiseOR($"colB"))
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def bitwiseOR(other: Any): Column = fn("|", other)
+
+ /**
+ * Compute bitwise AND of this expression with another expression.
+ * {{{
+ * df.select($"colA".bitwiseAND($"colB"))
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def bitwiseAND(other: Any): Column = fn("&", other)
+
+ /**
+ * Compute bitwise XOR of this expression with another expression.
+ * {{{
+ * df.select($"colA".bitwiseXOR($"colB"))
+ * }}}
+ *
+ * @group expr_ops
+ * @since 3.4.0
+ */
+ def bitwiseXOR(other: Any): Column = fn("^", other)
}
private[sql] object Column {
@@ -221,7 +1242,7 @@ private[sql] object Column {
case "*" =>
builder.getUnresolvedStarBuilder
case _ if name.endsWith(".*") =>
- unsupported("* with prefix is not supported yet.")
+ builder.getUnresolvedStarBuilder.setUnparsedTarget(name)
case _ =>
builder.getUnresolvedAttributeBuilder.setUnparsedIdentifier(name)
}
diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala
index fd0c9320ab4..7db44907111 100644
--- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -752,7 +752,7 @@ class Dataset[T] private[sql] (val session: SparkSession, private[sql] val plan:
session.newDataset { builder =>
builder.getSortBuilder
.setInput(plan.getRoot)
- .setIsGlobal(false)
+ .setIsGlobal(global)
.addAllOrder(sortExprs.map(_.sortOrder).asJava)
}
}
diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala
index 4b4bedaf659..7045b2885d3 100644
--- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala
+++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala
@@ -92,6 +92,34 @@ object functions {
}
}
+ /**
+ * Evaluates a list of conditions and returns one of multiple possible result expressions. If
+ * otherwise is not defined at the end, null is returned for unmatched conditions.
+ *
+ * {{{
+ * // Example: encoding gender string column into integer.
+ *
+ * // Scala:
+ * people.select(when(people("gender") === "male", 0)
+ * .when(people("gender") === "female", 1)
+ * .otherwise(2))
+ *
+ * // Java:
+ * people.select(when(col("gender").equalTo("male"), 0)
+ * .when(col("gender").equalTo("female"), 1)
+ * .otherwise(2))
+ * }}}
+ *
+ * @group normal_funcs
+ * @since 3.4.0
+ */
+ def when(condition: Column, value: Any): Column = Column { builder =>
+ builder.getUnresolvedFunctionBuilder
+ .setFunctionName("when")
+ .addArguments(condition.expr)
+ .addArguments(lit(value).expr)
+ }
+
/**
* Parses the expression string into the column that it represents, similar to
* [[Dataset#selectExpr]].
diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ColumnTestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ColumnTestSuite.scala
new file mode 100644
index 00000000000..d12f0747b86
--- /dev/null
+++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ColumnTestSuite.scala
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql
+
+import java.io.ByteArrayOutputStream
+
+import scala.collection.JavaConverters._
+
+import org.scalatest.funsuite.{AnyFunSuite => ConnectFunSuite} // scalastyle:ignore funsuite
+
+import org.apache.spark.sql.{functions => fn}
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Tests for client local Column behavior.
+ */
+class ColumnTestSuite extends ConnectFunSuite {
+ test("equals & hashcode") {
+ def expr: Column = fn.when(fn.col("a") < 10, "a").otherwise("b")
+ val a = expr
+ val b = expr
+ val c = expr.as("nope")
+ assert(a == a)
+ assert(b == b)
+ assert(c == c)
+ assert(a == b)
+ assert(b == a)
+ assert(a != c)
+ assert(c != a)
+ assert(b != c)
+ assert(c != b)
+ assert(a.hashCode == b.hashCode)
+ assert(a.hashCode != c.hashCode)
+ }
+
+ test("invalid when usage") {
+ intercept[IllegalArgumentException] {
+ fn.col("a").when(fn.lit(true), 2)
+ }
+ intercept[IllegalArgumentException] {
+ fn.col("a").isNull.when(fn.lit(true), 2)
+ }
+ intercept[IllegalArgumentException] {
+ fn.when(fn.col("a") < 10, 1)
+ .otherwise(2)
+ .when(fn.col("b") > 8, 3)
+ }
+ }
+
+ test("invalid otherwise usage") {
+ intercept[IllegalArgumentException] {
+ fn.col("a").otherwise(2)
+ }
+ intercept[IllegalArgumentException] {
+ fn.col("a").isNull.otherwise(2)
+ }
+ intercept[IllegalArgumentException] {
+ fn.when(fn.col("a") < 10, 1)
+ .otherwise(2)
+ .otherwise(3)
+ }
+ }
+
+ test("invalid withField usage") {
+ intercept[IllegalArgumentException] {
+ fn.col("c").withField(null, fn.lit(1))
+ }
+ intercept[IllegalArgumentException] {
+ fn.col("c").withField("x", null)
+ }
+ }
+
+ def testSame(
+ name: String,
+ f1: (Column, Column) => Column,
+ f2: (Column, Column) => Column): Unit = test(name + " are the same") {
+ val a = fn.col("a")
+ val b = fn.col("b")
+ assert(f1(a, b) == f2(a, b))
+ }
+ testSame("=== and equalTo", _ === _, _.equalTo(_))
+ testSame("=!= and notEqual", _ =!= _, _.notEqual(_))
+ testSame("> and gt", _ > _, _.gt(_))
+ testSame("< and lt", _ < _, _.lt(_))
+ testSame(">= and geq", _ >= _, _.geq(_))
+ testSame("<= and leq", _ <= _, _.leq(_))
+ testSame("<=> and eqNullSafe", _ <=> _, _.eqNullSafe(_))
+ testSame("|| and or", _ || _, _.or(_))
+ testSame("&& and and", _ && _, _.and(_))
+ testSame("+ and plus", _ + _, _.plus(_))
+ testSame("- and minus", _ - _, _.minus(_))
+ testSame("* and multiply", _ * _, _.multiply(_))
+ testSame("/ and divide", _ / _, _.divide(_))
+ testSame("% and mod", _ % _, _.mod(_))
+
+ test("isIn") {
+ val a = fn.col("a")
+ val values = Seq(1, 5, 6)
+ assert(a.isin(values: _*) == a.isInCollection(values))
+ assert(a.isin(values: _*) == a.isInCollection(values.asJava))
+ }
+
+ test("getItem/apply/getField are the same") {
+ val a = fn.col("a")
+ assert(a("x") == a.getItem("x"))
+ assert(a("x") == a.getField("x"))
+ }
+
+ test("substr variations") {
+ val a = fn.col("a")
+ assert(a.substr(2, 10) == a.substr(fn.lit(2), fn.lit(10)))
+ }
+
+ test("startsWith variations") {
+ val a = fn.col("a")
+ assert(a.endsWith("p_") == a.endsWith(fn.lit("p_")))
+ }
+
+ test("endsWith variations") {
+ val a = fn.col("a")
+ assert(a.endsWith("world") == a.endsWith(fn.lit("world")))
+ }
+
+ test("alias/as/name are the same") {
+ val a = fn.col("a")
+ assert(a.as("x") == a.alias("x"))
+ assert(a.as("x") == a.name("x"))
+ }
+
+ test("multi-alias variations") {
+ val a = fn.col("a")
+ assert(a.as("x" :: "y" :: Nil) == a.as(Array("x", "y")))
+ }
+
+ test("cast variations") {
+ val a = fn.col("a")
+ assert(a.cast("string") == a.cast(StringType))
+ }
+
+ test("desc and desc_nulls_last are the same") {
+ val a = fn.col("a")
+ assert(a.desc == a.desc_nulls_last)
+ }
+
+ test("asc and asc_nulls_first are the same") {
+ val a = fn.col("a")
+ assert(a.asc == a.asc_nulls_first)
+ }
+
+ private def captureStdOut(block: => Unit): String = {
+ val capturedOut = new ByteArrayOutputStream()
+ Console.withOut(capturedOut)(block)
+ capturedOut.toString()
+ }
+
+ test("explain") {
+ val x = fn.col("a") + fn.col("b")
+ val explain1 = captureStdOut(x.explain(false))
+ val explain2 = captureStdOut(x.explain(true))
+ assert(explain1 == explain2)
+ val expectedFragments = Seq("unresolved_function", "function_name: \"+\"", "arguments")
+ expectedFragments.foreach { fragment =>
+ assert(explain1.contains(fragment))
+ }
+ }
+}
diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
index 28ac12928fd..5a2a9485cf0 100644
--- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
+++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
@@ -31,7 +31,7 @@ import org.apache.spark.connect.proto
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{functions => fn}
import org.apache.spark.sql.connect.client.SparkConnectClient
-import org.apache.spark.sql.types.{MetadataBuilder, StructType}
+import org.apache.spark.sql.types.{MapType, MetadataBuilder, StringType, StructType}
// scalastyle:off
/**
@@ -153,25 +153,38 @@ class PlanGenerationTestSuite extends ConnectFunSuite with BeforeAndAfterAll wit
}
}
- private def createLocalRelation(schema: StructType): DataFrame = session.newDataset { builder =>
- // TODO API is not consistent. Now we have two different ways of working with schemas!
- builder.getLocalRelationBuilder.setSchema(schema.catalogString)
- }
-
private val simpleSchema = new StructType()
.add("id", "long")
.add("a", "int")
.add("b", "double")
+ private val simpleSchemaString = simpleSchema.catalogString
+
private val otherSchema = new StructType()
.add("a", "int")
.add("id", "long")
.add("payload", "binary")
+ private val otherSchemaString = otherSchema.catalogString
+
+ private val complexSchema = simpleSchema
+ .add("d", simpleSchema)
+ .add("e", "array<int>")
+ .add("f", MapType(StringType, simpleSchema))
+ .add("g", "string")
+
+ private val complexSchemaString = complexSchema.catalogString
+
+ private def createLocalRelation(schema: String): DataFrame = session.newDataset { builder =>
+ // TODO API is not consistent. Now we have two different ways of working with schemas!
+ builder.getLocalRelationBuilder.setSchema(schema)
+ }
+
// A few helper dataframes.
- private def simple: DataFrame = createLocalRelation(simpleSchema)
+ private def simple: DataFrame = createLocalRelation(simpleSchemaString)
private def left: DataFrame = simple
- private def right: DataFrame = createLocalRelation(otherSchema)
+ private def right: DataFrame = createLocalRelation(otherSchemaString)
+ private def complex = createLocalRelation(complexSchemaString)
private def select(cs: Column*): DataFrame = simple.select(cs: _*)
@@ -502,20 +515,215 @@ class PlanGenerationTestSuite extends ConnectFunSuite with BeforeAndAfterAll wit
}
/* Column API */
- test("column by name") {
- select(fn.col("b"))
+ private def columnTest(name: String)(f: => Column): Unit = {
+ test("column " + name) {
+ complex.select(f)
+ }
+ }
+
+ private def orderColumnTest(name: String)(f: => Column): Unit = {
+ test("column " + name) {
+ complex.orderBy(f)
+ }
+ }
+
+ columnTest("apply") {
+ fn.col("f").apply("super_duper_key")
+ }
+
+ columnTest("unary minus") {
+ -fn.lit(1)
+ }
+
+ columnTest("not") {
+ !fn.lit(true)
+ }
+
+ columnTest("equals") {
+ fn.col("a") === fn.col("b")
+ }
+
+ columnTest("not equals") {
+ fn.col("a") =!= fn.col("b")
+ }
+
+ columnTest("gt") {
+ fn.col("a") > fn.col("b")
+ }
+
+ columnTest("lt") {
+ fn.col("a") < fn.col("b")
+ }
+
+ columnTest("geq") {
+ fn.col("a") >= fn.col("b")
+ }
+
+ columnTest("leq") {
+ fn.col("a") <= fn.col("b")
+ }
+
+ columnTest("eqNullSafe") {
+ fn.col("a") <=> fn.col("b")
+ }
+
+ columnTest("when otherwise") {
+ val a = fn.col("a")
+ fn.when(a < 10, "low").when(a < 20, "medium").otherwise("high")
+ }
+
+ columnTest("between") {
+ fn.col("a").between(10, 20)
+ }
+
+ columnTest("isNaN") {
+ fn.col("b").isNaN
+ }
+
+ columnTest("isNull") {
+ fn.col("g").isNull
+ }
+
+ columnTest("isNotNull") {
+ fn.col("g").isNotNull
+ }
+
+ columnTest("and") {
+ fn.col("a") > 10 && fn.col("b") < 0.5d
+ }
+
+ columnTest("or") {
+ fn.col("a") > 10 || fn.col("b") < 0.5d
+ }
+
+ columnTest("add") {
+ fn.col("a") + fn.col("b")
+ }
+
+ columnTest("subtract") {
+ fn.col("a") - fn.col("b")
+ }
+
+ columnTest("multiply") {
+ fn.col("a") * fn.col("b")
+ }
+
+ columnTest("divide") {
+ fn.col("a") / fn.col("b")
+ }
+
+ columnTest("modulo") {
+ fn.col("a") % 10
+ }
+
+ columnTest("isin") {
+ fn.col("g").isin("hello", "world", "foo")
+ }
+
+ columnTest("like") {
+ fn.col("g").like("%bob%")
+ }
+
+ columnTest("rlike") {
+ fn.col("g").like("^[0-9]*$")
+ }
+
+ columnTest("ilike") {
+ fn.col("g").like("%fOb%")
+ }
+
+ columnTest("getItem") {
+ fn.col("e").getItem(3)
+ }
+
+ columnTest("withField") {
+ fn.col("d").withField("x", fn.lit("xq"))
+ }
+
+ columnTest("dropFields") {
+ fn.col("d").dropFields("a", "c")
+ }
+
+ columnTest("getField") {
+ fn.col("d").getItem("b")
+ }
+
+ columnTest("substr") {
+ fn.col("g").substr(8, 3)
+ }
+
+ columnTest("contains") {
+ fn.col("g").contains("baz")
+ }
+
+ columnTest("startsWith") {
+ fn.col("g").startsWith("prefix_")
+ }
+
+ columnTest("endsWith") {
+ fn.col("g").endsWith("suffix_")
+ }
+
+ columnTest("alias") {
+ fn.col("a").name("b")
+ }
+
+ columnTest("as multi") {
+ fn.col("d").as(Array("v1", "v2", "v3"))
+ }
+
+ columnTest("as with metadata") {
+ val builder = new MetadataBuilder
+ builder.putString("comment", "modified C field")
+ fn.col("c").as("c_mod", builder.build())
+ }
+
+ columnTest("cast") {
+ fn.col("a").cast("long")
+ }
+
+ orderColumnTest("desc") {
+ fn.col("b").desc
+ }
+
+ orderColumnTest("desc_nulls_first") {
+ fn.col("b").desc_nulls_first
+ }
+
+ orderColumnTest("desc_nulls_last") {
+ fn.col("b").desc_nulls_last
+ }
+
+ orderColumnTest("asc") {
+ fn.col("a").asc
+ }
+
+ orderColumnTest("asc_nulls_first") {
+ fn.col("a").asc_nulls_first
+ }
+
+ orderColumnTest("asc_nulls_last") {
+ fn.col("a").asc_nulls_last
+ }
+
+ columnTest("bitwiseOR") {
+ fn.col("a").bitwiseOR(7)
+ }
+
+ columnTest("bitwiseAND") {
+ fn.col("a").bitwiseAND(255)
}
- test("column add") {
- select(fn.col("a") + fn.col("b"))
+ columnTest("bitwiseXOR") {
+ fn.col("a").bitwiseXOR(78)
}
- test("column alias") {
- select(fn.col("a").name("b"))
+ columnTest("star") {
+ fn.col("*")
}
- test("column equals") {
- select(fn.col("a") === fn.col("b"))
+ columnTest("star with target") {
+ fn.col("str.*")
}
/* Function API */
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_add.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_add.explain
index 3a5d916ea9d..7a4e1d8c99e 100644
--- a/connector/connect/common/src/test/resources/query-tests/explain-results/column_add.explain
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_add.explain
@@ -1,2 +1,2 @@
'Project [unresolvedalias('`+`('a, 'b), None)]
-+- LocalRelation <empty>, [none#0L, none#1, none#2]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_alias.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_alias.explain
index ef8d28db6a5..6599480d99f 100644
--- a/connector/connect/common/src/test/resources/query-tests/explain-results/column_alias.explain
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_alias.explain
@@ -1,2 +1,2 @@
'Project ['a AS #0]
-+- LocalRelation <empty>, [none#0L, none#1, none#2]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_and.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_and.explain
new file mode 100644
index 00000000000..f5a189da6f8
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_and.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('and('`>`('a, 10), '`<`('b, 0.5)), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_apply.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_apply.explain
new file mode 100644
index 00000000000..8ae42a493d8
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_apply.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('f[super_duper_key], None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_as_multi.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_as_multi.explain
new file mode 100644
index 00000000000..0afa06a3c1b
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_as_multi.explain
@@ -0,0 +1,2 @@
+'Project ['d AS Buffer(v1, v2, v3)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_as_with_metadata.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_as_with_metadata.explain
new file mode 100644
index 00000000000..6e03801f2fe
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_as_with_metadata.explain
@@ -0,0 +1,2 @@
+'Project ['c AS #0]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_asc.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_asc.explain
new file mode 100644
index 00000000000..d1538809a01
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_asc.explain
@@ -0,0 +1,2 @@
+'Sort ['a ASC NULLS FIRST], true
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_asc_nulls_first.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_asc_nulls_first.explain
new file mode 100644
index 00000000000..d1538809a01
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_asc_nulls_first.explain
@@ -0,0 +1,2 @@
+'Sort ['a ASC NULLS FIRST], true
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_asc_nulls_last.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_asc_nulls_last.explain
new file mode 100644
index 00000000000..3d8ad392e46
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_asc_nulls_last.explain
@@ -0,0 +1,2 @@
+'Sort ['a ASC NULLS LAST], true
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_between.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_between.explain
new file mode 100644
index 00000000000..af895c4fa0d
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_between.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('and('`>=`('a, 10), '`<=`('a, 20)), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_bitwiseAND.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_bitwiseAND.explain
new file mode 100644
index 00000000000..49063410a41
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_bitwiseAND.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('`&`('a, 255), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_bitwiseOR.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_bitwiseOR.explain
new file mode 100644
index 00000000000..73fc02c35d8
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_bitwiseOR.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('`|`('a, 7), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_bitwiseXOR.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_bitwiseXOR.explain
new file mode 100644
index 00000000000..37edb68a75f
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_bitwiseXOR.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('`^`('a, 78), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_by_name.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_by_name.explain
deleted file mode 100644
index 24a45fce0e8..00000000000
--- a/connector/connect/common/src/test/resources/query-tests/explain-results/column_by_name.explain
+++ /dev/null
@@ -1,2 +0,0 @@
-'Project ['b]
-+- LocalRelation <empty>, [none#0L, none#1, none#2]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_cast.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_cast.explain
new file mode 100644
index 00000000000..3587da106aa
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_cast.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias(cast('a as bigint), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_contains.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_contains.explain
new file mode 100644
index 00000000000..3cf63e745a6
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_contains.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('contains('g, baz), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_desc.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_desc.explain
new file mode 100644
index 00000000000..70b7479554a
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_desc.explain
@@ -0,0 +1,2 @@
+'Sort ['b DESC NULLS LAST], true
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_desc_nulls_first.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_desc_nulls_first.explain
new file mode 100644
index 00000000000..372b32017ca
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_desc_nulls_first.explain
@@ -0,0 +1,2 @@
+'Sort ['b DESC NULLS FIRST], true
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_desc_nulls_last.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_desc_nulls_last.explain
new file mode 100644
index 00000000000..70b7479554a
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_desc_nulls_last.explain
@@ -0,0 +1,2 @@
+'Sort ['b DESC NULLS LAST], true
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_divide.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_divide.explain
new file mode 100644
index 00000000000..08f4f8e393e
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_divide.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('`/`('a, 'b), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_dropFields.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_dropFields.explain
new file mode 100644
index 00000000000..ad5585fcf3c
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_dropFields.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias(update_fields(update_fields('d, dropfield(a)), dropfield(c)), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_endsWith.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_endsWith.explain
new file mode 100644
index 00000000000..75a9e6fa9f2
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_endsWith.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('endswith('g, suffix_), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_eqNullSafe.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_eqNullSafe.explain
new file mode 100644
index 00000000000..39d14ed9827
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_eqNullSafe.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('`<=>`('a, 'b), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_equals.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_equals.explain
index 6980b810213..2f1eab15525 100644
--- a/connector/connect/common/src/test/resources/query-tests/explain-results/column_equals.explain
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_equals.explain
@@ -1,2 +1,2 @@
'Project [unresolvedalias('`=`('a, 'b), None)]
-+- LocalRelation <empty>, [none#0L, none#1, none#2]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_geq.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_geq.explain
new file mode 100644
index 00000000000..37e3352670b
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_geq.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('`>=`('a, 'b), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_getField.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_getField.explain
new file mode 100644
index 00000000000..230bd2fb598
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_getField.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('d[b], None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_getItem.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_getItem.explain
new file mode 100644
index 00000000000..b0e6e9ccf78
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_getItem.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('e[3], None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_gt.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_gt.explain
new file mode 100644
index 00000000000..38cc79bbe7e
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_gt.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('`>`('a, 'b), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_ilike.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_ilike.explain
new file mode 100644
index 00000000000..588ef3a5b22
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_ilike.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('like('g, %fOb%), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_isNaN.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_isNaN.explain
new file mode 100644
index 00000000000..8fcb852a6b1
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_isNaN.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('isNaN('b), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_isNotNull.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_isNotNull.explain
new file mode 100644
index 00000000000..ec0497b2591
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_isNotNull.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('isNotNull('g), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_isNull.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_isNull.explain
new file mode 100644
index 00000000000..b1a24bf5f28
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_isNull.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('isNull('g), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_isin.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_isin.explain
new file mode 100644
index 00000000000..ea92861186c
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_isin.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('g IN (hello,world,foo), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_leq.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_leq.explain
new file mode 100644
index 00000000000..01fcaee9ba0
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_leq.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('`<=`('a, 'b), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_like.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_like.explain
new file mode 100644
index 00000000000..54c8d50eabf
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_like.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('like('g, %bob%), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_lt.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_lt.explain
new file mode 100644
index 00000000000..2438cf3ea4e
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_lt.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('`<`('a, 'b), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_modulo.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_modulo.explain
new file mode 100644
index 00000000000..d010d609b84
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_modulo.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('`%`('a, 10), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_multiply.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_multiply.explain
new file mode 100644
index 00000000000..3bb266b609f
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_multiply.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('`*`('a, 'b), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_not.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_not.explain
new file mode 100644
index 00000000000..897371ed75c
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_not.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('`!`(true), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_not_equals.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_not_equals.explain
new file mode 100644
index 00000000000..7ebf39127e8
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_not_equals.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('`!`('`=`('a, 'b)), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_or.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_or.explain
new file mode 100644
index 00000000000..a5e77e36639
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_or.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('or('`>`('a, 10), '`<`('b, 0.5)), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_rlike.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_rlike.explain
new file mode 100644
index 00000000000..8cf1f42bcab
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_rlike.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('like('g, ^[0-9]*$), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_star.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_star.explain
new file mode 100644
index 00000000000..a9384942aa1
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_star.explain
@@ -0,0 +1,2 @@
+'Project [*]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_star_with_target.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_star_with_target.explain
new file mode 100644
index 00000000000..116393cfb02
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_star_with_target.explain
@@ -0,0 +1,2 @@
+'Project [ArrayBuffer(str).*]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_startsWith.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_startsWith.explain
new file mode 100644
index 00000000000..bdf95dba324
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_startsWith.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('startswith('g, prefix_), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_substr.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_substr.explain
new file mode 100644
index 00000000000..f3e081be5db
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_substr.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('substr('g, 8, 3), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_subtract.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_subtract.explain
new file mode 100644
index 00000000000..eb56ecad484
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_subtract.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('`-`('a, 'b), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_unary_minus.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_unary_minus.explain
new file mode 100644
index 00000000000..95354071643
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_unary_minus.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias('negative(1), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_when_otherwise.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_when_otherwise.explain
new file mode 100644
index 00000000000..8f972122b13
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_when_otherwise.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias(CASE WHEN '`<`('a, 10) THEN low WHEN '`<`('a, 20) THEN medium ELSE high END, None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/column_withField.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/column_withField.explain
new file mode 100644
index 00000000000..386554d6e47
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/column_withField.explain
@@ -0,0 +1,2 @@
+'Project [unresolvedalias(update_fields('d, WithField(x, xq)), None)]
++- LocalRelation <empty>, [none#0L, none#1, none#2, none#3, none#4, none#5, none#6]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/orderBy_columns.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/orderBy_columns.explain
index f183bc7be36..59017198796 100644
--- a/connector/connect/common/src/test/resources/query-tests/explain-results/orderBy_columns.explain
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/orderBy_columns.explain
@@ -1,2 +1,2 @@
-'Sort ['id ASC NULLS FIRST, 'b ASC NULLS FIRST, 'a ASC NULLS FIRST], false
+'Sort ['id ASC NULLS FIRST, 'b ASC NULLS FIRST, 'a ASC NULLS FIRST], true
+- LocalRelation <empty>, [none#0L, none#1, none#2]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/orderBy_strings.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/orderBy_strings.explain
index c976a65aab5..3b005e5a1dd 100644
--- a/connector/connect/common/src/test/resources/query-tests/explain-results/orderBy_strings.explain
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/orderBy_strings.explain
@@ -1,2 +1,2 @@
-'Sort ['b ASC NULLS FIRST, 'id ASC NULLS FIRST, 'a ASC NULLS FIRST], false
+'Sort ['b ASC NULLS FIRST, 'id ASC NULLS FIRST, 'a ASC NULLS FIRST], true
+- LocalRelation <empty>, [none#0L, none#1, none#2]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/sort_columns.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/sort_columns.explain
index 79ae0d702aa..ce5e8ea1f8d 100644
--- a/connector/connect/common/src/test/resources/query-tests/explain-results/sort_columns.explain
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/sort_columns.explain
@@ -1,2 +1,2 @@
-'Sort ['id ASC NULLS FIRST, 'b ASC NULLS FIRST], false
+'Sort ['id ASC NULLS FIRST, 'b ASC NULLS FIRST], true
+- LocalRelation <empty>, [none#0L, none#1, none#2]
diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/sort_strings.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/sort_strings.explain
index 7bc8b68fe37..96c18b836c1 100644
--- a/connector/connect/common/src/test/resources/query-tests/explain-results/sort_strings.explain
+++ b/connector/connect/common/src/test/resources/query-tests/explain-results/sort_strings.explain
@@ -1,2 +1,2 @@
-'Sort ['b ASC NULLS FIRST, 'a ASC NULLS FIRST], false
+'Sort ['b ASC NULLS FIRST, 'a ASC NULLS FIRST], true
+- LocalRelation <empty>, [none#0L, none#1, none#2]
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json b/connector/connect/common/src/test/resources/query-tests/queries/column_add.json
index 69cdae5f503..c7c88bdc5d1 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_add.json
@@ -2,7 +2,7 @@
"project": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"expressions": [{
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_add.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_add.proto.bin
index 0a260449e59..26ccd472276 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_add.proto.bin
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_add.proto.bin
@@ -1,5 +1,5 @@
-;
-$Z" struct<id:bigint,a:int,b:double>
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+
a
b
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_alias.json b/connector/connect/common/src/test/resources/query-tests/queries/column_alias.json
index 3a8d1e54a39..559d7c7b254 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_alias.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_alias.json
@@ -2,7 +2,7 @@
"project": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"expressions": [{
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_alias.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_alias.proto.bin
index 92add9d5d42..b05b217b061 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_alias.proto.bin
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_alias.proto.bin
@@ -1,5 +1,5 @@
-4
-$Z" struct<id:bigint,a:int,b:double>2
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>2
ab
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_and.json b/connector/connect/common/src/test/resources/query-tests/queries/column_and.json
new file mode 100644
index 00000000000..1faa3965b12
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_and.json
@@ -0,0 +1,41 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "and",
+ "arguments": [{
+ "unresolvedFunction": {
+ "functionName": "\u003e",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "a"
+ }
+ }, {
+ "literal": {
+ "integer": 10
+ }
+ }]
+ }
+ }, {
+ "unresolvedFunction": {
+ "functionName": "\u003c",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "b"
+ }
+ }, {
+ "literal": {
+ "double": 0.5
+ }
+ }]
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_and.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_and.proto.bin
new file mode 100644
index 00000000000..a566569f293
Binary files /dev/null and b/connector/connect/common/src/test/resources/query-tests/queries/column_and.proto.bin differ
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_apply.json b/connector/connect/common/src/test/resources/query-tests/queries/column_apply.json
new file mode 100644
index 00000000000..c72cdda0c29
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_apply.json
@@ -0,0 +1,23 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedExtractValue": {
+ "child": {
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "f"
+ }
+ },
+ "extraction": {
+ "literal": {
+ "string": "super_duper_key"
+ }
+ }
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_apply.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_apply.proto.bin
new file mode 100644
index 00000000000..bbc3bacbd9c
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_apply.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>b
+
+f
+jsuper_duper_key
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_as_multi.json b/connector/connect/common/src/test/resources/query-tests/queries/column_as_multi.json
new file mode 100644
index 00000000000..940f6a5139c
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_as_multi.json
@@ -0,0 +1,19 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "alias": {
+ "expr": {
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "d"
+ }
+ },
+ "name": ["v1", "v2", "v3"]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_as_multi.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_as_multi.proto.bin
new file mode 100644
index 00000000000..a8568f73f8d
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_as_multi.proto.bin
@@ -0,0 +1,4 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>2
+
+dv1v2v3
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_as_with_metadata.json b/connector/connect/common/src/test/resources/query-tests/queries/column_as_with_metadata.json
new file mode 100644
index 00000000000..6c399f9959e
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_as_with_metadata.json
@@ -0,0 +1,20 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "alias": {
+ "expr": {
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "c"
+ }
+ },
+ "name": ["c_mod"],
+ "metadata": "{\"comment\":\"modified C field\"}"
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_as_with_metadata.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_as_with_metadata.proto.bin
new file mode 100644
index 00000000000..65161a7a307
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_as_with_metadata.proto.bin
@@ -0,0 +1,4 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>02.
+
+cc_mod{"comment":"modified C field"}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.json b/connector/connect/common/src/test/resources/query-tests/queries/column_asc.json
similarity index 50%
copy from connector/connect/common/src/test/resources/query-tests/queries/sort_strings.json
copy to connector/connect/common/src/test/resources/query-tests/queries/column_asc.json
index 16220ca0947..11d2915091c 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_asc.json
@@ -2,18 +2,10 @@
"sort": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"order": [{
- "child": {
- "unresolvedAttribute": {
- "unparsedIdentifier": "b"
- }
- },
- "direction": "SORT_DIRECTION_ASCENDING",
- "nullOrdering": "SORT_NULLS_FIRST"
- }, {
"child": {
"unresolvedAttribute": {
"unparsedIdentifier": "a"
@@ -22,6 +14,6 @@
"direction": "SORT_DIRECTION_ASCENDING",
"nullOrdering": "SORT_NULLS_FIRST"
}],
- "isGlobal": false
+ "isGlobal": true
}
}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_asc.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_asc.proto.bin
new file mode 100644
index 00000000000..56e5c5d2076
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_asc.proto.bin
@@ -0,0 +1,4 @@
+:�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+
+a
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.json b/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_first.json
similarity index 50%
copy from connector/connect/common/src/test/resources/query-tests/queries/sort_strings.json
copy to connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_first.json
index 16220ca0947..11d2915091c 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_first.json
@@ -2,18 +2,10 @@
"sort": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"order": [{
- "child": {
- "unresolvedAttribute": {
- "unparsedIdentifier": "b"
- }
- },
- "direction": "SORT_DIRECTION_ASCENDING",
- "nullOrdering": "SORT_NULLS_FIRST"
- }, {
"child": {
"unresolvedAttribute": {
"unparsedIdentifier": "a"
@@ -22,6 +14,6 @@
"direction": "SORT_DIRECTION_ASCENDING",
"nullOrdering": "SORT_NULLS_FIRST"
}],
- "isGlobal": false
+ "isGlobal": true
}
}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_first.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_first.proto.bin
new file mode 100644
index 00000000000..56e5c5d2076
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_first.proto.bin
@@ -0,0 +1,4 @@
+:�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+
+a
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_last.json b/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_last.json
new file mode 100644
index 00000000000..eac9be9e5ac
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_last.json
@@ -0,0 +1,19 @@
+{
+ "sort": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "order": [{
+ "child": {
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "a"
+ }
+ },
+ "direction": "SORT_DIRECTION_ASCENDING",
+ "nullOrdering": "SORT_NULLS_LAST"
+ }],
+ "isGlobal": true
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_last.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_last.proto.bin
new file mode 100644
index 00000000000..21665330de2
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_last.proto.bin
@@ -0,0 +1,4 @@
+:�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+
+a
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_between.json b/connector/connect/common/src/test/resources/query-tests/queries/column_between.json
new file mode 100644
index 00000000000..8473a3e7cda
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_between.json
@@ -0,0 +1,41 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "and",
+ "arguments": [{
+ "unresolvedFunction": {
+ "functionName": "\u003e\u003d",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "a"
+ }
+ }, {
+ "literal": {
+ "integer": 10
+ }
+ }]
+ }
+ }, {
+ "unresolvedFunction": {
+ "functionName": "\u003c\u003d",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "a"
+ }
+ }, {
+ "literal": {
+ "integer": 20
+ }
+ }]
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_between.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_between.proto.bin
new file mode 100644
index 00000000000..c651f8856bb
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_between.proto.bin
@@ -0,0 +1,10 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>1/
+and
+>=
+a
+0
+
+<=
+a
+0
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.json b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.json
new file mode 100644
index 00000000000..17431e74d94
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.json
@@ -0,0 +1,23 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "\u0026",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "a"
+ }
+ }, {
+ "literal": {
+ "integer": 255
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.proto.bin
new file mode 100644
index 00000000000..3517777d8af
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+&
+a
+0�
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseOR.json
similarity index 50%
copy from connector/connect/common/src/test/resources/query-tests/queries/column_add.json
copy to connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseOR.json
index 69cdae5f503..cfd8e2dfb7f 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseOR.json
@@ -2,19 +2,19 @@
"project": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"expressions": [{
"unresolvedFunction": {
- "functionName": "+",
+ "functionName": "|",
"arguments": [{
"unresolvedAttribute": {
"unparsedIdentifier": "a"
}
}, {
- "unresolvedAttribute": {
- "unparsedIdentifier": "b"
+ "literal": {
+ "integer": 7
}
}]
}
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseOR.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseOR.proto.bin
new file mode 100644
index 00000000000..16ea227007b
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseOR.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+|
+a
+0
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.json
similarity index 50%
copy from connector/connect/common/src/test/resources/query-tests/queries/column_add.json
copy to connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.json
index 69cdae5f503..d3e4c326c26 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.json
@@ -2,19 +2,19 @@
"project": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"expressions": [{
"unresolvedFunction": {
- "functionName": "+",
+ "functionName": "^",
"arguments": [{
"unresolvedAttribute": {
"unparsedIdentifier": "a"
}
}, {
- "unresolvedAttribute": {
- "unparsedIdentifier": "b"
+ "literal": {
+ "integer": 78
}
}]
}
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.proto.bin
new file mode 100644
index 00000000000..2c154418bb7
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+^
+a
+0N
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_by_name.json b/connector/connect/common/src/test/resources/query-tests/queries/column_by_name.json
deleted file mode 100644
index 19e00da08be..00000000000
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_by_name.json
+++ /dev/null
@@ -1,14 +0,0 @@
-{
- "project": {
- "input": {
- "localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
- }
- },
- "expressions": [{
- "unresolvedAttribute": {
- "unparsedIdentifier": "b"
- }
- }]
- }
-}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_by_name.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_by_name.proto.bin
deleted file mode 100644
index 0a70dc0f88c..00000000000
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_by_name.proto.bin
+++ /dev/null
@@ -1,3 +0,0 @@
--
-$Z" struct<id:bigint,a:int,b:double>
-b
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_cast.json b/connector/connect/common/src/test/resources/query-tests/queries/column_cast.json
new file mode 100644
index 00000000000..d014ce98803
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_cast.json
@@ -0,0 +1,22 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "cast": {
+ "expr": {
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "a"
+ }
+ },
+ "type": {
+ "long": {
+ }
+ }
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_cast.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_cast.proto.bin
new file mode 100644
index 00000000000..3b3d04193ac
Binary files /dev/null and b/connector/connect/common/src/test/resources/query-tests/queries/column_cast.proto.bin differ
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_contains.json b/connector/connect/common/src/test/resources/query-tests/queries/column_contains.json
new file mode 100644
index 00000000000..862dd8cefa0
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_contains.json
@@ -0,0 +1,23 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "contains",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "g"
+ }
+ }, {
+ "literal": {
+ "string": "baz"
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_contains.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_contains.proto.bin
new file mode 100644
index 00000000000..12a45de9ce6
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_contains.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+contains
+g
+jbaz
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_desc.json b/connector/connect/common/src/test/resources/query-tests/queries/column_desc.json
new file mode 100644
index 00000000000..f882e4abad2
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_desc.json
@@ -0,0 +1,19 @@
+{
+ "sort": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "order": [{
+ "child": {
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "b"
+ }
+ },
+ "direction": "SORT_DIRECTION_DESCENDING",
+ "nullOrdering": "SORT_NULLS_LAST"
+ }],
+ "isGlobal": true
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_desc.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_desc.proto.bin
new file mode 100644
index 00000000000..3d9b64634d9
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_desc.proto.bin
@@ -0,0 +1,4 @@
+:�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+
+b
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_first.json b/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_first.json
new file mode 100644
index 00000000000..707ced41a16
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_first.json
@@ -0,0 +1,19 @@
+{
+ "sort": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "order": [{
+ "child": {
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "b"
+ }
+ },
+ "direction": "SORT_DIRECTION_DESCENDING",
+ "nullOrdering": "SORT_NULLS_FIRST"
+ }],
+ "isGlobal": true
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_first.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_first.proto.bin
new file mode 100644
index 00000000000..54ae15ef70b
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_first.proto.bin
@@ -0,0 +1,4 @@
+:�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+
+b
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_last.json b/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_last.json
new file mode 100644
index 00000000000..f882e4abad2
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_last.json
@@ -0,0 +1,19 @@
+{
+ "sort": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "order": [{
+ "child": {
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "b"
+ }
+ },
+ "direction": "SORT_DIRECTION_DESCENDING",
+ "nullOrdering": "SORT_NULLS_LAST"
+ }],
+ "isGlobal": true
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_last.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_last.proto.bin
new file mode 100644
index 00000000000..3d9b64634d9
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_last.proto.bin
@@ -0,0 +1,4 @@
+:�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+
+b
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json b/connector/connect/common/src/test/resources/query-tests/queries/column_divide.json
similarity index 60%
copy from connector/connect/common/src/test/resources/query-tests/queries/column_add.json
copy to connector/connect/common/src/test/resources/query-tests/queries/column_divide.json
index 69cdae5f503..08171761822 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_divide.json
@@ -2,12 +2,12 @@
"project": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"expressions": [{
"unresolvedFunction": {
- "functionName": "+",
+ "functionName": "/",
"arguments": [{
"unresolvedAttribute": {
"unparsedIdentifier": "a"
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_divide.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_divide.proto.bin
new file mode 100644
index 00000000000..856a7565591
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_divide.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+/
+a
+b
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_dropFields.json b/connector/connect/common/src/test/resources/query-tests/queries/column_dropFields.json
new file mode 100644
index 00000000000..ff39dfa6532
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_dropFields.json
@@ -0,0 +1,24 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "updateFields": {
+ "structExpression": {
+ "updateFields": {
+ "structExpression": {
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "d"
+ }
+ },
+ "fieldName": "a"
+ }
+ },
+ "fieldName": "c"
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_dropFields.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_dropFields.proto.bin
new file mode 100644
index 00000000000..b95d8d3dcd6
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_dropFields.proto.bin
@@ -0,0 +1,6 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>j
+j
+
+
+dac
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_endsWith.json b/connector/connect/common/src/test/resources/query-tests/queries/column_endsWith.json
new file mode 100644
index 00000000000..99edefdc9a4
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_endsWith.json
@@ -0,0 +1,23 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "endswith",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "g"
+ }
+ }, {
+ "literal": {
+ "string": "suffix_"
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_endsWith.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_endsWith.proto.bin
new file mode 100644
index 00000000000..c854a66c726
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_endsWith.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+endswith
+g
+ jsuffix_
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json b/connector/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.json
similarity index 58%
copy from connector/connect/common/src/test/resources/query-tests/queries/column_add.json
copy to connector/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.json
index 69cdae5f503..3481ee46e5a 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.json
@@ -2,12 +2,12 @@
"project": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"expressions": [{
"unresolvedFunction": {
- "functionName": "+",
+ "functionName": "\u003c\u003d\u003e",
"arguments": [{
"unresolvedAttribute": {
"unparsedIdentifier": "a"
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.proto.bin
new file mode 100644
index 00000000000..d38e1836b3f
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+<=>
+a
+b
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_equals.json b/connector/connect/common/src/test/resources/query-tests/queries/column_equals.json
index 5817a3e26e6..834afc4dee7 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_equals.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_equals.json
@@ -2,7 +2,7 @@
"project": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"expressions": [{
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_equals.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_equals.proto.bin
index 2e78b895111..e8505e38fdc 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_equals.proto.bin
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_equals.proto.bin
@@ -1,5 +1,5 @@
-;
-$Z" struct<id:bigint,a:int,b:double>
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
=
a
b
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json b/connector/connect/common/src/test/resources/query-tests/queries/column_geq.json
similarity index 59%
copy from connector/connect/common/src/test/resources/query-tests/queries/column_add.json
copy to connector/connect/common/src/test/resources/query-tests/queries/column_geq.json
index 69cdae5f503..aca63fadf09 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_geq.json
@@ -2,12 +2,12 @@
"project": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"expressions": [{
"unresolvedFunction": {
- "functionName": "+",
+ "functionName": "\u003e\u003d",
"arguments": [{
"unresolvedAttribute": {
"unparsedIdentifier": "a"
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_geq.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_geq.proto.bin
new file mode 100644
index 00000000000..c0762cef053
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_geq.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+>=
+a
+b
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_getField.json b/connector/connect/common/src/test/resources/query-tests/queries/column_getField.json
new file mode 100644
index 00000000000..d68d274a406
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_getField.json
@@ -0,0 +1,23 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedExtractValue": {
+ "child": {
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "d"
+ }
+ },
+ "extraction": {
+ "literal": {
+ "string": "b"
+ }
+ }
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_getField.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_getField.proto.bin
new file mode 100644
index 00000000000..eac146d122f
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_getField.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>b
+
+d
+jb
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_getItem.json b/connector/connect/common/src/test/resources/query-tests/queries/column_getItem.json
new file mode 100644
index 00000000000..41eb3eadafc
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_getItem.json
@@ -0,0 +1,23 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedExtractValue": {
+ "child": {
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "e"
+ }
+ },
+ "extraction": {
+ "literal": {
+ "integer": 3
+ }
+ }
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_getItem.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_getItem.proto.bin
new file mode 100644
index 00000000000..92083231b14
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_getItem.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>b
+
+e
+0
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json b/connector/connect/common/src/test/resources/query-tests/queries/column_gt.json
similarity index 59%
copy from connector/connect/common/src/test/resources/query-tests/queries/column_add.json
copy to connector/connect/common/src/test/resources/query-tests/queries/column_gt.json
index 69cdae5f503..658a17c00ab 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_gt.json
@@ -2,12 +2,12 @@
"project": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"expressions": [{
"unresolvedFunction": {
- "functionName": "+",
+ "functionName": "\u003e",
"arguments": [{
"unresolvedAttribute": {
"unparsedIdentifier": "a"
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_gt.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_gt.proto.bin
new file mode 100644
index 00000000000..9304b3dae72
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_gt.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+>
+a
+b
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_ilike.json b/connector/connect/common/src/test/resources/query-tests/queries/column_ilike.json
new file mode 100644
index 00000000000..bc751f0ce1d
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_ilike.json
@@ -0,0 +1,23 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "like",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "g"
+ }
+ }, {
+ "literal": {
+ "string": "%fOb%"
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_ilike.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_ilike.proto.bin
new file mode 100644
index 00000000000..10226e50caa
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_ilike.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+like
+g
+j%fOb%
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json b/connector/connect/common/src/test/resources/query-tests/queries/column_isNaN.json
similarity index 52%
copy from connector/connect/common/src/test/resources/query-tests/queries/column_add.json
copy to connector/connect/common/src/test/resources/query-tests/queries/column_isNaN.json
index 69cdae5f503..0eb722a08ec 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_isNaN.json
@@ -2,17 +2,13 @@
"project": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"expressions": [{
"unresolvedFunction": {
- "functionName": "+",
+ "functionName": "isNaN",
"arguments": [{
- "unresolvedAttribute": {
- "unparsedIdentifier": "a"
- }
- }, {
"unresolvedAttribute": {
"unparsedIdentifier": "b"
}
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_isNaN.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_isNaN.proto.bin
new file mode 100644
index 00000000000..1f96b82d070
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_isNaN.proto.bin
@@ -0,0 +1,4 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+isNaN
+b
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_isNotNull.json b/connector/connect/common/src/test/resources/query-tests/queries/column_isNotNull.json
new file mode 100644
index 00000000000..bb2403f1d19
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_isNotNull.json
@@ -0,0 +1,19 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "isNotNull",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "g"
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_isNotNull.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_isNotNull.proto.bin
new file mode 100644
index 00000000000..45574baf87c
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_isNotNull.proto.bin
@@ -0,0 +1,4 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+ isNotNull
+g
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_isNull.json b/connector/connect/common/src/test/resources/query-tests/queries/column_isNull.json
new file mode 100644
index 00000000000..386ad8898eb
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_isNull.json
@@ -0,0 +1,19 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "isNull",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "g"
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_isNull.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_isNull.proto.bin
new file mode 100644
index 00000000000..44b48593580
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_isNull.proto.bin
@@ -0,0 +1,4 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+isNull
+g
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_isin.json b/connector/connect/common/src/test/resources/query-tests/queries/column_isin.json
new file mode 100644
index 00000000000..1a990cc0851
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_isin.json
@@ -0,0 +1,31 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "in",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "g"
+ }
+ }, {
+ "literal": {
+ "string": "hello"
+ }
+ }, {
+ "literal": {
+ "string": "world"
+ }
+ }, {
+ "literal": {
+ "string": "foo"
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_isin.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_isin.proto.bin
new file mode 100644
index 00000000000..9e61c9d26d6
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_isin.proto.bin
@@ -0,0 +1,7 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>,*
+in
+g
+jhello
+jworld
+jfoo
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json b/connector/connect/common/src/test/resources/query-tests/queries/column_leq.json
similarity index 59%
copy from connector/connect/common/src/test/resources/query-tests/queries/column_add.json
copy to connector/connect/common/src/test/resources/query-tests/queries/column_leq.json
index 69cdae5f503..9ea4d99f9b7 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_leq.json
@@ -2,12 +2,12 @@
"project": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"expressions": [{
"unresolvedFunction": {
- "functionName": "+",
+ "functionName": "\u003c\u003d",
"arguments": [{
"unresolvedAttribute": {
"unparsedIdentifier": "a"
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_leq.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_leq.proto.bin
new file mode 100644
index 00000000000..7f62d95ee34
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_leq.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+<=
+a
+b
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_like.json b/connector/connect/common/src/test/resources/query-tests/queries/column_like.json
new file mode 100644
index 00000000000..2afab4b0329
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_like.json
@@ -0,0 +1,23 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "like",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "g"
+ }
+ }, {
+ "literal": {
+ "string": "%bob%"
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_like.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_like.proto.bin
new file mode 100644
index 00000000000..dee7e79301c
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_like.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+like
+g
+j%bob%
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json b/connector/connect/common/src/test/resources/query-tests/queries/column_lt.json
similarity index 59%
copy from connector/connect/common/src/test/resources/query-tests/queries/column_add.json
copy to connector/connect/common/src/test/resources/query-tests/queries/column_lt.json
index 69cdae5f503..019e33cd106 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_lt.json
@@ -2,12 +2,12 @@
"project": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"expressions": [{
"unresolvedFunction": {
- "functionName": "+",
+ "functionName": "\u003c",
"arguments": [{
"unresolvedAttribute": {
"unparsedIdentifier": "a"
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_lt.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_lt.proto.bin
new file mode 100644
index 00000000000..ff658957cee
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_lt.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+<
+a
+b
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json b/connector/connect/common/src/test/resources/query-tests/queries/column_modulo.json
similarity index 50%
copy from connector/connect/common/src/test/resources/query-tests/queries/column_add.json
copy to connector/connect/common/src/test/resources/query-tests/queries/column_modulo.json
index 69cdae5f503..7cbe8a11fd9 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_modulo.json
@@ -2,19 +2,19 @@
"project": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"expressions": [{
"unresolvedFunction": {
- "functionName": "+",
+ "functionName": "%",
"arguments": [{
"unresolvedAttribute": {
"unparsedIdentifier": "a"
}
}, {
- "unresolvedAttribute": {
- "unparsedIdentifier": "b"
+ "literal": {
+ "integer": 10
}
}]
}
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_modulo.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_modulo.proto.bin
new file mode 100644
index 00000000000..461b09b0b82
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_modulo.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+%
+a
+0
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json b/connector/connect/common/src/test/resources/query-tests/queries/column_multiply.json
similarity index 60%
copy from connector/connect/common/src/test/resources/query-tests/queries/column_add.json
copy to connector/connect/common/src/test/resources/query-tests/queries/column_multiply.json
index 69cdae5f503..58ebb2604da 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_multiply.json
@@ -2,12 +2,12 @@
"project": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"expressions": [{
"unresolvedFunction": {
- "functionName": "+",
+ "functionName": "*",
"arguments": [{
"unresolvedAttribute": {
"unparsedIdentifier": "a"
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_multiply.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_multiply.proto.bin
new file mode 100644
index 00000000000..4ca350de10b
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_multiply.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+*
+a
+b
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_not.json b/connector/connect/common/src/test/resources/query-tests/queries/column_not.json
new file mode 100644
index 00000000000..39417760083
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_not.json
@@ -0,0 +1,19 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "!",
+ "arguments": [{
+ "literal": {
+ "boolean": true
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_not.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_not.proto.bin
new file mode 100644
index 00000000000..59a14a40198
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_not.proto.bin
@@ -0,0 +1,4 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+!
+
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_not_equals.json b/connector/connect/common/src/test/resources/query-tests/queries/column_not_equals.json
new file mode 100644
index 00000000000..de8f07fb478
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_not_equals.json
@@ -0,0 +1,28 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "!",
+ "arguments": [{
+ "unresolvedFunction": {
+ "functionName": "\u003d",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "a"
+ }
+ }, {
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "b"
+ }
+ }]
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_not_equals.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_not_equals.proto.bin
new file mode 100644
index 00000000000..006450915a3
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_not_equals.proto.bin
@@ -0,0 +1,6 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+!
+=
+a
+b
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_or.json b/connector/connect/common/src/test/resources/query-tests/queries/column_or.json
new file mode 100644
index 00000000000..091280b2ae6
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_or.json
@@ -0,0 +1,41 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "or",
+ "arguments": [{
+ "unresolvedFunction": {
+ "functionName": "\u003e",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "a"
+ }
+ }, {
+ "literal": {
+ "integer": 10
+ }
+ }]
+ }
+ }, {
+ "unresolvedFunction": {
+ "functionName": "\u003c",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "b"
+ }
+ }, {
+ "literal": {
+ "double": 0.5
+ }
+ }]
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_or.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_or.proto.bin
new file mode 100644
index 00000000000..aaf78e293e9
Binary files /dev/null and b/connector/connect/common/src/test/resources/query-tests/queries/column_or.proto.bin differ
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_rlike.json b/connector/connect/common/src/test/resources/query-tests/queries/column_rlike.json
new file mode 100644
index 00000000000..490766a88d7
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_rlike.json
@@ -0,0 +1,23 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "like",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "g"
+ }
+ }, {
+ "literal": {
+ "string": "^[0-9]*$"
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_rlike.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_rlike.proto.bin
new file mode 100644
index 00000000000..7e2b6996c70
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_rlike.proto.bin
@@ -0,0 +1,6 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+like
+g
+
+j^[0-9]*$
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_star.json b/connector/connect/common/src/test/resources/query-tests/queries/column_star.json
new file mode 100644
index 00000000000..ecb84fee174
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_star.json
@@ -0,0 +1,13 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedStar": {
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_star.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_star.proto.bin
new file mode 100644
index 00000000000..bde97f5f8ef
Binary files /dev/null and b/connector/connect/common/src/test/resources/query-tests/queries/column_star.proto.bin differ
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_star_with_target.json b/connector/connect/common/src/test/resources/query-tests/queries/column_star_with_target.json
new file mode 100644
index 00000000000..2b873369b9d
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_star_with_target.json
@@ -0,0 +1,14 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedStar": {
+ "unparsedTarget": "str.*"
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_star_with_target.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_star_with_target.proto.bin
new file mode 100644
index 00000000000..a15c03b48b9
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_star_with_target.proto.bin
@@ -0,0 +1,3 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string> *
+str.*
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_startsWith.json b/connector/connect/common/src/test/resources/query-tests/queries/column_startsWith.json
new file mode 100644
index 00000000000..fe66f718e23
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_startsWith.json
@@ -0,0 +1,23 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "startswith",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "g"
+ }
+ }, {
+ "literal": {
+ "string": "prefix_"
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_startsWith.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_startsWith.proto.bin
new file mode 100644
index 00000000000..dd10f51fcdd
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_startsWith.proto.bin
@@ -0,0 +1,6 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>"
+
+startswith
+g
+ jprefix_
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_substr.json b/connector/connect/common/src/test/resources/query-tests/queries/column_substr.json
new file mode 100644
index 00000000000..6c074decbba
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_substr.json
@@ -0,0 +1,27 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "substr",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "g"
+ }
+ }, {
+ "literal": {
+ "integer": 8
+ }
+ }, {
+ "literal": {
+ "integer": 3
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_substr.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_substr.proto.bin
new file mode 100644
index 00000000000..e050495220d
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_substr.proto.bin
@@ -0,0 +1,6 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+substr
+g
+0
+0
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json b/connector/connect/common/src/test/resources/query-tests/queries/column_subtract.json
similarity index 60%
copy from connector/connect/common/src/test/resources/query-tests/queries/column_add.json
copy to connector/connect/common/src/test/resources/query-tests/queries/column_subtract.json
index 69cdae5f503..79a42858f09 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_subtract.json
@@ -2,12 +2,12 @@
"project": {
"input": {
"localRelation": {
- "schema": "struct\u003cid:bigint,a:int,b:double\u003e"
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
}
},
"expressions": [{
"unresolvedFunction": {
- "functionName": "+",
+ "functionName": "-",
"arguments": [{
"unresolvedAttribute": {
"unparsedIdentifier": "a"
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_subtract.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_subtract.proto.bin
new file mode 100644
index 00000000000..794972c95bf
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_subtract.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+-
+a
+b
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_unary_minus.json b/connector/connect/common/src/test/resources/query-tests/queries/column_unary_minus.json
new file mode 100644
index 00000000000..5e39ebe6074
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_unary_minus.json
@@ -0,0 +1,19 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "negative",
+ "arguments": [{
+ "literal": {
+ "integer": 1
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_unary_minus.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_unary_minus.proto.bin
new file mode 100644
index 00000000000..7ff0a7c8ad5
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_unary_minus.proto.bin
@@ -0,0 +1,4 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>
+negative
+0
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.json b/connector/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.json
new file mode 100644
index 00000000000..8c0004a6abb
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.json
@@ -0,0 +1,53 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "when",
+ "arguments": [{
+ "unresolvedFunction": {
+ "functionName": "\u003c",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "a"
+ }
+ }, {
+ "literal": {
+ "integer": 10
+ }
+ }]
+ }
+ }, {
+ "literal": {
+ "string": "low"
+ }
+ }, {
+ "unresolvedFunction": {
+ "functionName": "\u003c",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "a"
+ }
+ }, {
+ "literal": {
+ "integer": 20
+ }
+ }]
+ }
+ }, {
+ "literal": {
+ "string": "medium"
+ }
+ }, {
+ "literal": {
+ "string": "high"
+ }
+ }]
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.proto.bin
new file mode 100644
index 00000000000..51e8c7ce37e
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.proto.bin
@@ -0,0 +1,14 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>OM
+when
+<
+a
+0
+
+jlow
+<
+a
+0
+
+jmedium
+jhigh
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_withField.json b/connector/connect/common/src/test/resources/query-tests/queries/column_withField.json
new file mode 100644
index 00000000000..ba9d22bcc5d
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_withField.json
@@ -0,0 +1,24 @@
+{
+ "project": {
+ "input": {
+ "localRelation": {
+ "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e"
+ }
+ },
+ "expressions": [{
+ "updateFields": {
+ "structExpression": {
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "d"
+ }
+ },
+ "fieldName": "x",
+ "valueExpression": {
+ "literal": {
+ "string": "xq"
+ }
+ }
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_withField.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_withField.proto.bin
new file mode 100644
index 00000000000..33be23b4c61
--- /dev/null
+++ b/connector/connect/common/src/test/resources/query-tests/queries/column_withField.proto.bin
@@ -0,0 +1,5 @@
+�
+�Z��struct<id:bigint,a:int,b:double,d:struct<id:bigint,a:int,b:double>,e:array<int>,f:map<string,struct<id:bigint,a:int,b:double>>,g:string>j
+
+dx
+jxq
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/orderBy_columns.json b/connector/connect/common/src/test/resources/query-tests/queries/orderBy_columns.json
index 84cb594f6a1..d935be14ee6 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/orderBy_columns.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/orderBy_columns.json
@@ -30,6 +30,6 @@
"direction": "SORT_DIRECTION_ASCENDING",
"nullOrdering": "SORT_NULLS_FIRST"
}],
- "isGlobal": false
+ "isGlobal": true
}
}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/orderBy_columns.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/orderBy_columns.proto.bin
index 4061563400c..bebe09e64a9 100644
Binary files a/connector/connect/common/src/test/resources/query-tests/queries/orderBy_columns.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/orderBy_columns.proto.bin differ
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/orderBy_strings.json b/connector/connect/common/src/test/resources/query-tests/queries/orderBy_strings.json
index 41bf89273e2..22508b999d3 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/orderBy_strings.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/orderBy_strings.json
@@ -30,6 +30,6 @@
"direction": "SORT_DIRECTION_ASCENDING",
"nullOrdering": "SORT_NULLS_FIRST"
}],
- "isGlobal": false
+ "isGlobal": true
}
}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/orderBy_strings.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/orderBy_strings.proto.bin
index e2f5ccbdd9a..3e7b74ea6b5 100644
Binary files a/connector/connect/common/src/test/resources/query-tests/queries/orderBy_strings.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/orderBy_strings.proto.bin differ
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sort_columns.json b/connector/connect/common/src/test/resources/query-tests/queries/sort_columns.json
index 08d98935f40..65354a13130 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/sort_columns.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/sort_columns.json
@@ -22,6 +22,6 @@
"direction": "SORT_DIRECTION_ASCENDING",
"nullOrdering": "SORT_NULLS_FIRST"
}],
- "isGlobal": false
+ "isGlobal": true
}
}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sort_columns.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/sort_columns.proto.bin
index 5c076fa9d12..f3f7a40246b 100644
Binary files a/connector/connect/common/src/test/resources/query-tests/queries/sort_columns.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/sort_columns.proto.bin differ
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.json b/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.json
index 16220ca0947..6d127fa5c3c 100644
--- a/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.json
+++ b/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.json
@@ -22,6 +22,6 @@
"direction": "SORT_DIRECTION_ASCENDING",
"nullOrdering": "SORT_NULLS_FIRST"
}],
- "isGlobal": false
+ "isGlobal": true
}
}
\ No newline at end of file
diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.proto.bin
index 01a61b9c158..1c7568f4f34 100644
Binary files a/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.proto.bin differ
diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/dsl/package.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/dsl/package.scala
index 3e301bb8823..fd6790cead7 100644
--- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/dsl/package.scala
+++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/dsl/package.scala
@@ -210,7 +210,7 @@ package object dsl {
mode
.map(SaveMode.valueOf(_))
- .map(SaveModeConverter.toSaveModeProto(_))
+ .map(SaveModeConverter.toSaveModeProto)
.foreach(writeOp.setMode(_))
if (tableName.nonEmpty) {
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org