Class

org.apache.spark.sql

Column

Related Doc: package sql

Permalink

class Column extends Logging

A column that will be computed based on the data in a DataFrame.

A new column is constructed based on the input columns present in a dataframe:

df("columnName")            // On a specific DataFrame.
col("columnName")           // A generic column no yet associated with a DataFrame.
col("columnName.field")     // Extracting a struct field
col("`a.column.with.dots`") // Escape `.` in column names.
$"columnName"               // Scala short hand for a named column.
expr("a + 1")               // A column that is constructed from a parsed SQL Expression.
lit("abc")                  // A column that produces a literal (constant) value.

Column objects can be composed to form complex expressions:

$"a" + 1
$"a" === $"b"
Source
Column.scala
Since

1.3.0

Linear Supertypes
Logging, AnyRef, Any
Known Subclasses
Ordering
  1. Grouped
  2. Alphabetic
  3. By Inheritance
Inherited
  1. Column
  2. Logging
  3. AnyRef
  4. Any
  1. Hide All
  2. Show All
Visibility
  1. Public
  2. All

Instance Constructors

  1. new Column(name: String)

    Permalink
  2. new Column(expr: Expression)

    Permalink

Value Members

  1. final def !=(arg0: Any): Boolean

    Permalink
    Definition Classes
    AnyRef → Any
  2. final def ##(): Int

    Permalink
    Definition Classes
    AnyRef → Any
  3. def %(other: Any): Column

    Permalink

    Modulo (a.k.a.

    Modulo (a.k.a. remainder) expression.

    Since

    1.3.0

  4. def &&(other: Any): Column

    Permalink

    Boolean AND.

    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("inSchool").and(people("isEmployed")) );
    Since

    1.3.0

  5. def *(other: Any): Column

    Permalink

    Multiplication of this expression and another expression.

    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("height").multiply(people("weight")) );
    Since

    1.3.0

  6. def +(other: Any): Column

    Permalink

    Sum of this expression and another expression.

    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("height").plus(people("weight")) );
    Since

    1.3.0

  7. def -(other: Any): Column

    Permalink

    Subtraction.

    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("height").minus(people("weight")) );
    Since

    1.3.0

  8. def /(other: Any): Column

    Permalink

    Division this expression by another expression.

    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("height").divide(people("weight")) );
    Since

    1.3.0

  9. def <(other: Any): Column

    Permalink

    Less than.

    Less than.

    // Scala: The following selects people younger than 21.
    people.select( people("age") < 21 )
    
    // Java:
    people.select( people("age").lt(21) );
    Since

    1.3.0

  10. def <=(other: Any): Column

    Permalink

    Less than or equal to.

    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("age").leq(21) );
    Since

    1.3.0

  11. def <=>(other: Any): Column

    Permalink

    Equality test that is safe for null values.

    Equality test that is safe for null values.

    Since

    1.3.0

  12. def =!=(other: Any): Column

    Permalink

    Inequality test.

    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")) );
    Since

    2.0.0

  13. final def ==(arg0: Any): Boolean

    Permalink
    Definition Classes
    AnyRef → Any
  14. def ===(other: Any): Column

    Permalink

    Equality test.

    Equality test.

    // Scala:
    df.filter( df("colA") === df("colB") )
    
    // Java
    import static org.apache.spark.sql.functions.*;
    df.filter( col("colA").equalTo(col("colB")) );
    Since

    1.3.0

  15. def >(other: Any): Column

    Permalink

    Greater than.

    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("age").gt(21) );
    Since

    1.3.0

  16. def >=(other: Any): Column

    Permalink

    Greater than or equal to an expression.

    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("age").geq(21) )
    Since

    1.3.0

  17. def alias(alias: String): Column

    Permalink

    Gives the column an alias.

    Gives the column an alias. Same as as.

    // Renames colA to colB in select output.
    df.select($"colA".alias("colB"))
    Since

    1.4.0

  18. def and(other: Column): Column

    Permalink

    Boolean AND.

    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("inSchool").and(people("isEmployed")) );
    Since

    1.3.0

  19. def apply(extraction: Any): Column

    Permalink

    Extracts a value or values from a complex type.

    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
    Since

    1.4.0

  20. def as(alias: String, metadata: Metadata): Column

    Permalink

    Gives the column an alias with metadata.

    Gives the column an alias with metadata.

    val metadata: Metadata = ...
    df.select($"colA".as("colB", metadata))
    Since

    1.3.0

  21. def as(alias: Symbol): Column

    Permalink

    Gives the column an 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 as with explicitly empty metadata.

    Since

    1.3.0

  22. def as(aliases: Array[String]): Column

    Permalink

    Assigns the given aliases to the results of a table generating function.

    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))
    Since

    1.4.0

  23. def as(aliases: Seq[String]): Column

    Permalink

    (Scala-specific) Assigns the given aliases to the results of a table generating function.

    (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))
    Since

    1.4.0

  24. def as(alias: String): Column

    Permalink

    Gives the column an 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 as with explicitly empty metadata.

    Since

    1.3.0

  25. def as[U](implicit arg0: Encoder[U]): TypedColumn[Any, U]

    Permalink

    Provides a type hint about the expected return value of this column.

    Provides a type hint about the expected return value of this column. This information can be used by operations such as select on a Dataset to automatically convert the results into the correct JVM types.

    Since

    1.6.0

  26. final def asInstanceOf[T0]: T0

    Permalink
    Definition Classes
    Any
  27. def asc: Column

    Permalink

    Returns an ordering used in sorting.

    Returns an ordering used in sorting.

    // Scala: sort a DataFrame by age column in ascending order.
    df.sort(df("age").asc)
    
    // Java
    df.sort(df.col("age").asc());
    Since

    1.3.0

  28. def between(lowerBound: Any, upperBound: Any): Column

    Permalink

    True if the current column is between the lower bound and upper bound, inclusive.

    True if the current column is between the lower bound and upper bound, inclusive.

    Since

    1.4.0

  29. def bitwiseAND(other: Any): Column

    Permalink

    Compute bitwise AND of this expression with another expression.

    Compute bitwise AND of this expression with another expression.

    df.select($"colA".bitwiseAND($"colB"))
    Since

    1.4.0

  30. def bitwiseOR(other: Any): Column

    Permalink

    Compute bitwise OR of this expression with another expression.

    Compute bitwise OR of this expression with another expression.

    df.select($"colA".bitwiseOR($"colB"))
    Since

    1.4.0

  31. def bitwiseXOR(other: Any): Column

    Permalink

    Compute bitwise XOR of this expression with another expression.

    Compute bitwise XOR of this expression with another expression.

    df.select($"colA".bitwiseXOR($"colB"))
    Since

    1.4.0

  32. def cast(to: String): Column

    Permalink

    Casts the column to a different data type, using the canonical string representation of the type.

    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"))
    Since

    1.3.0

  33. def cast(to: DataType): Column

    Permalink

    Casts the column to a different data type.

    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"))
    Since

    1.3.0

  34. def clone(): AnyRef

    Permalink
    Attributes
    protected[java.lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  35. def contains(other: Any): Column

    Permalink

    Contains the other element.

    Contains the other element.

    Since

    1.3.0

  36. def desc: Column

    Permalink

    Returns an ordering used in sorting.

    Returns an ordering used in sorting.

    // Scala: sort a DataFrame by age column in descending order.
    df.sort(df("age").desc)
    
    // Java
    df.sort(df.col("age").desc());
    Since

    1.3.0

  37. def divide(other: Any): Column

    Permalink

    Division this expression by another expression.

    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("height").divide(people("weight")) );
    Since

    1.3.0

  38. def endsWith(literal: String): Column

    Permalink

    String ends with another string literal.

    String ends with another string literal.

    Since

    1.3.0

  39. def endsWith(other: Column): Column

    Permalink

    String ends with.

    String ends with.

    Since

    1.3.0

  40. final def eq(arg0: AnyRef): Boolean

    Permalink
    Definition Classes
    AnyRef
  41. def eqNullSafe(other: Any): Column

    Permalink

    Equality test that is safe for null values.

    Equality test that is safe for null values.

    Since

    1.3.0

  42. def equalTo(other: Any): Column

    Permalink

    Equality test.

    Equality test.

    // Scala:
    df.filter( df("colA") === df("colB") )
    
    // Java
    import static org.apache.spark.sql.functions.*;
    df.filter( col("colA").equalTo(col("colB")) );
    Since

    1.3.0

  43. def equals(that: Any): Boolean

    Permalink
    Definition Classes
    Column → AnyRef → Any
  44. def explain(extended: Boolean): Unit

    Permalink

    Prints the expression to the console for debugging purpose.

    Prints the expression to the console for debugging purpose.

    Since

    1.3.0

  45. val expr: Expression

    Permalink
    Attributes
    protected[org.apache.spark.sql]
  46. def finalize(): Unit

    Permalink
    Attributes
    protected[java.lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( classOf[java.lang.Throwable] )
  47. def geq(other: Any): Column

    Permalink

    Greater than or equal to an expression.

    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("age").geq(21) )
    Since

    1.3.0

  48. final def getClass(): Class[_]

    Permalink
    Definition Classes
    AnyRef → Any
  49. def getField(fieldName: String): Column

    Permalink

    An expression that gets a field by name in a StructType.

    An expression that gets a field by name in a StructType.

    Since

    1.3.0

  50. def getItem(key: Any): Column

    Permalink

    An expression that gets an item at position ordinal out of an array, or gets a value by key key in a MapType.

    An expression that gets an item at position ordinal out of an array, or gets a value by key key in a MapType.

    Since

    1.3.0

  51. def gt(other: Any): Column

    Permalink

    Greater than.

    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("age").gt(21) );
    Since

    1.3.0

  52. def hashCode(): Int

    Permalink
    Definition Classes
    Column → AnyRef → Any
  53. def initializeLogIfNecessary(isInterpreter: Boolean): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  54. final def isInstanceOf[T0]: Boolean

    Permalink
    Definition Classes
    Any
  55. def isNaN: Column

    Permalink

    True if the current expression is NaN.

    True if the current expression is NaN.

    Since

    1.5.0

  56. def isNotNull: Column

    Permalink

    True if the current expression is NOT null.

    True if the current expression is NOT null.

    Since

    1.3.0

  57. def isNull: Column

    Permalink

    True if the current expression is null.

    True if the current expression is null.

    Since

    1.3.0

  58. def isTraceEnabled(): Boolean

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  59. def isin(list: Any*): Column

    Permalink

    A boolean expression that is evaluated to true if the value of this expression is contained by the evaluated values of the arguments.

    A boolean expression that is evaluated to true if the value of this expression is contained by the evaluated values of the arguments.

    Annotations
    @varargs()
    Since

    1.5.0

  60. def leq(other: Any): Column

    Permalink

    Less than or equal to.

    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("age").leq(21) );
    Since

    1.3.0

  61. def like(literal: String): Column

    Permalink

    SQL like expression.

    SQL like expression.

    Since

    1.3.0

  62. def log: Logger

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  63. def logDebug(msg: ⇒ String, throwable: Throwable): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  64. def logDebug(msg: ⇒ String): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  65. def logError(msg: ⇒ String, throwable: Throwable): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  66. def logError(msg: ⇒ String): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  67. def logInfo(msg: ⇒ String, throwable: Throwable): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  68. def logInfo(msg: ⇒ String): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  69. def logName: String

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  70. def logTrace(msg: ⇒ String, throwable: Throwable): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  71. def logTrace(msg: ⇒ String): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  72. def logWarning(msg: ⇒ String, throwable: Throwable): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  73. def logWarning(msg: ⇒ String): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  74. def lt(other: Any): Column

    Permalink

    Less than.

    Less than.

    // Scala: The following selects people younger than 21.
    people.select( people("age") < 21 )
    
    // Java:
    people.select( people("age").lt(21) );
    Since

    1.3.0

  75. def minus(other: Any): Column

    Permalink

    Subtraction.

    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("height").minus(people("weight")) );
    Since

    1.3.0

  76. def mod(other: Any): Column

    Permalink

    Modulo (a.k.a.

    Modulo (a.k.a. remainder) expression.

    Since

    1.3.0

  77. def multiply(other: Any): Column

    Permalink

    Multiplication of this expression and another expression.

    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("height").multiply(people("weight")) );
    Since

    1.3.0

  78. def name(alias: String): Column

    Permalink

    Gives the column a name (alias).

    Gives the column a name (alias).

    // Renames colA to colB in select output.
    df.select($"colA".name("colB"))

    If the current column has metadata associated with it, this metadata will be propagated to the new column. If this not desired, use as with explicitly empty metadata.

    Since

    2.0.0

  79. final def ne(arg0: AnyRef): Boolean

    Permalink
    Definition Classes
    AnyRef
  80. def notEqual(other: Any): Column

    Permalink

    Inequality test.

    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")) );
    Since

    1.3.0

  81. final def notify(): Unit

    Permalink
    Definition Classes
    AnyRef
  82. final def notifyAll(): Unit

    Permalink
    Definition Classes
    AnyRef
  83. def or(other: Column): Column

    Permalink

    Boolean OR.

    Boolean OR.

    // Scala: The following selects people that are in school or employed.
    people.filter( people("inSchool") || people("isEmployed") )
    
    // Java:
    people.filter( people("inSchool").or(people("isEmployed")) );
    Since

    1.3.0

  84. def otherwise(value: Any): Column

    Permalink

    Evaluates a list of conditions and returns one of multiple possible result expressions.

    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))
    Since

    1.4.0

  85. def over(): Column

    Permalink

    Define a empty analytic clause.

    Define a empty analytic clause. In this case the analytic function is applied and presented for all rows in the result set.

    df.select(
      sum("price").over(),
      avg("price").over()
    )
    Since

    2.0.0

  86. def over(window: WindowSpec): Column

    Permalink

    Define a windowing column.

    Define a windowing column.

    val w = Window.partitionBy("name").orderBy("id")
    df.select(
      sum("price").over(w.rangeBetween(Long.MinValue, 2)),
      avg("price").over(w.rowsBetween(0, 4))
    )
    Since

    1.4.0

  87. def plus(other: Any): Column

    Permalink

    Sum of this expression and another expression.

    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("height").plus(people("weight")) );
    Since

    1.3.0

  88. def rlike(literal: String): Column

    Permalink

    SQL RLIKE expression (LIKE with Regex).

    SQL RLIKE expression (LIKE with Regex).

    Since

    1.3.0

  89. def startsWith(literal: String): Column

    Permalink

    String starts with another string literal.

    String starts with another string literal.

    Since

    1.3.0

  90. def startsWith(other: Column): Column

    Permalink

    String starts with.

    String starts with.

    Since

    1.3.0

  91. def substr(startPos: Int, len: Int): Column

    Permalink

    An expression that returns a substring.

    An expression that returns a substring.

    startPos

    starting position.

    len

    length of the substring.

    Since

    1.3.0

  92. def substr(startPos: Column, len: Column): Column

    Permalink

    An expression that returns a substring.

    An expression that returns a substring.

    startPos

    expression for the starting position.

    len

    expression for the length of the substring.

    Since

    1.3.0

  93. final def synchronized[T0](arg0: ⇒ T0): T0

    Permalink
    Definition Classes
    AnyRef
  94. def toString(): String

    Permalink
    Definition Classes
    Column → AnyRef → Any
  95. def unary_!: Column

    Permalink

    Inversion of boolean expression, i.e.

    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")) );
    Since

    1.3.0

  96. def unary_-: Column

    Permalink

    Unary minus, i.e.

    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") );
    Since

    1.3.0

  97. final def wait(): Unit

    Permalink
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  98. final def wait(arg0: Long, arg1: Int): Unit

    Permalink
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  99. final def wait(arg0: Long): Unit

    Permalink
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  100. def when(condition: Column, value: Any): Column

    Permalink

    Evaluates a list of conditions and returns one of multiple possible result expressions.

    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))
    Since

    1.4.0

  101. def ||(other: Any): Column

    Permalink

    Boolean OR.

    Boolean OR.

    // Scala: The following selects people that are in school or employed.
    people.filter( people("inSchool") || people("isEmployed") )
    
    // Java:
    people.filter( people("inSchool").or(people("isEmployed")) );
    Since

    1.3.0

Deprecated Value Members

  1. def !==(other: Any): Column

    Permalink

    Inequality test.

    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")) );
    Annotations
    @deprecated
    Deprecated

    (Since version 2.0.0) !== does not have the same precedence as ===, use =!= instead

    Since

    1.3.0

Inherited from Logging

Inherited from AnyRef

Inherited from Any

DataFrame functions

Expression operators

Java-specific expression operators

Support functions for DataFrames