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:44:24 UTC

[spark] branch branch-3.4 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 branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
     new e724f1e0466 [SPARK-42441][CONNECT] Scala Client add Column APIs
e724f1e0466 is described below

commit e724f1e0466e7b26a575587a7bb74f42cc54c420
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>
    (cherry picked from commit d4a5beb45b8a6fbde3b7d252fcd83755cef97aa3)
    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