class Column extends Logging
A column that will be computed based on the data in a DataFrame
.
A new column can be constructed based on the input columns present in a DataFrame:
df("columnName") // On a specific `df` DataFrame. col("columnName") // A generic column not 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.
Column objects can be composed to form complex expressions:
$"a" + 1 $"a" === $"b"
- Annotations
- @Stable()
- Source
- Column.scala
- Since
1.3.0
- Note
The internal Catalyst expression can be accessed via expr, but this method is for debugging purposes only and can change in any future Spark releases.
- Grouped
- Alphabetic
- By Inheritance
- Column
- Logging
- AnyRef
- Any
- Hide All
- Show All
- Public
- Protected
Type Members
- implicit class LogStringContext extends AnyRef
- Definition Classes
- Logging
Value Members
- final def !=(arg0: Any): Boolean
- Definition Classes
- AnyRef → Any
- final def ##: Int
- Definition Classes
- AnyRef → Any
- def %(other: Any): Column
Modulo (a.k.a.
Modulo (a.k.a. remainder) expression.
- Since
1.3.0
- def &&(other: Any): Column
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.col("inSchool").and(people.col("isEmployed")) );
- Since
1.3.0
- def *(other: Any): Column
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.col("height").multiply(people.col("weight")) );
- Since
1.3.0
- def +(other: Any): Column
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.col("height").plus(people.col("weight")) );
- Since
1.3.0
- def -(other: Any): Column
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.col("height").minus(people.col("weight")) );
- Since
1.3.0
- def /(other: Any): Column
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.col("height").divide(people.col("weight")) );
- Since
1.3.0
- def <(other: Any): Column
Less than.
Less than.
// Scala: The following selects people younger than 21. people.select( people("age") < 21 ) // Java: people.select( people.col("age").lt(21) );
- Since
1.3.0
- def <=(other: Any): Column
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.col("age").leq(21) );
- Since
1.3.0
- def <=>(other: Any): Column
Equality test that is safe for null values.
Equality test that is safe for null values.
- Since
1.3.0
- def =!=(other: Any): Column
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
- final def ==(arg0: Any): Boolean
- Definition Classes
- AnyRef → Any
- def ===(other: Any): Column
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
- def >(other: Any): Column
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.col("age").gt(21) );
- Since
1.3.0
- def >=(other: Any): Column
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.col("age").geq(21) )
- Since
1.3.0
- def alias(alias: String): Column
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
- def and(other: Column): Column
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.col("inSchool").and(people.col("isEmployed")) );
- Since
1.3.0
- def apply(extraction: Any): Column
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
- def as(alias: String, metadata: Metadata): Column
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
- def as(alias: Symbol): Column
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 the API
as(alias: String, metadata: Metadata)
with explicit metadata.- Since
1.3.0
- def as(aliases: Array[String]): Column
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
- def as(aliases: Seq[String]): Column
(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
- def as(alias: String): Column
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 the API
as(alias: String, metadata: Metadata)
with explicit metadata.- Since
1.3.0
- def as[U](implicit arg0: Encoder[U]): TypedColumn[Any, U]
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
- final def asInstanceOf[T0]: T0
- Definition Classes
- Any
- def asc: Column
Returns a sort expression based on ascending order of the column.
Returns a sort expression based on ascending order of the column.
// 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
- def asc_nulls_first: Column
Returns a sort expression based on ascending order of the column, and null values return before non-null values.
Returns a sort expression based on ascending order of the column, and null values return before non-null values.
// Scala: sort a DataFrame by age column in ascending order and null values appearing first. df.sort(df("age").asc_nulls_first) // Java df.sort(df.col("age").asc_nulls_first());
- Since
2.1.0
- def asc_nulls_last: Column
Returns a sort expression based on ascending order of the column, and null values appear after non-null values.
Returns a sort expression based on ascending order of the column, and null values appear after non-null values.
// Scala: sort a DataFrame by age column in ascending order and null values appearing last. df.sort(df("age").asc_nulls_last) // Java df.sort(df.col("age").asc_nulls_last());
- Since
2.1.0
- def between(lowerBound: Any, upperBound: Any): Column
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
- def bitwiseAND(other: Any): Column
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
- def bitwiseOR(other: Any): Column
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
- def bitwiseXOR(other: Any): Column
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
- def cast(to: String): Column
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
- def cast(to: DataType): Column
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
- def clone(): AnyRef
- Attributes
- protected[lang]
- Definition Classes
- AnyRef
- Annotations
- @throws(classOf[java.lang.CloneNotSupportedException]) @IntrinsicCandidate() @native()
- def contains(other: Any): Column
Contains the other element.
Contains the other element. Returns a boolean column based on a string match.
- Since
1.3.0
- def desc: Column
Returns a sort expression based on the descending order of the column.
Returns a sort expression based on the descending order of the column.
// Scala df.sort(df("age").desc) // Java df.sort(df.col("age").desc());
- Since
1.3.0
- def desc_nulls_first: Column
Returns a sort expression based on the descending order of the column, and null values appear before non-null values.
Returns a sort expression based on the descending order of the column, and null values appear before non-null values.
// Scala: sort a DataFrame by age column in descending order and null values appearing first. df.sort(df("age").desc_nulls_first) // Java df.sort(df.col("age").desc_nulls_first());
- Since
2.1.0
- def desc_nulls_last: Column
Returns a sort expression based on the descending order of the column, and null values appear after non-null values.
Returns a sort expression based on the descending order of the column, and null values appear after non-null values.
// Scala: sort a DataFrame by age column in descending order and null values appearing last. df.sort(df("age").desc_nulls_last) // Java df.sort(df.col("age").desc_nulls_last());
- Since
2.1.0
- def divide(other: Any): Column
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.col("height").divide(people.col("weight")) );
- Since
1.3.0
- def dropFields(fieldNames: String*): Column
An expression that drops fields in
StructType
by name.An expression that drops fields in
StructType
by name. This is a no-op if schema doesn't contain field name(s).val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col") df.select($"struct_col".dropFields("b")) // result: {"a":1} 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}}
- Since
3.1.0
- def endsWith(literal: String): Column
String ends with another string literal.
String ends with another string literal. Returns a boolean column based on a string match.
- Since
1.3.0
- def endsWith(other: Column): Column
String ends with.
String ends with. Returns a boolean column based on a string match.
- Since
1.3.0
- final def eq(arg0: AnyRef): Boolean
- Definition Classes
- AnyRef
- def eqNullSafe(other: Any): Column
Equality test that is safe for null values.
Equality test that is safe for null values.
- Since
1.3.0
- def equalTo(other: Any): Column
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
- def equals(that: Any): Boolean
- Definition Classes
- Column → AnyRef → Any
- def explain(extended: Boolean): Unit
Prints the expression to the console for debugging purposes.
Prints the expression to the console for debugging purposes.
- Since
1.3.0
- val expr: Expression
- def geq(other: Any): Column
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.col("age").geq(21) )
- Since
1.3.0
- final def getClass(): Class[_ <: AnyRef]
- Definition Classes
- AnyRef → Any
- Annotations
- @IntrinsicCandidate() @native()
- def getField(fieldName: String): Column
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
- def getItem(key: Any): Column
An expression that gets an item at position
ordinal
out of an array, or gets a value by keykey
in aMapType
.An expression that gets an item at position
ordinal
out of an array, or gets a value by keykey
in aMapType
.- Since
1.3.0
- def gt(other: Any): Column
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.col("age").gt(21) );
- Since
1.3.0
- def hashCode(): Int
- Definition Classes
- Column → AnyRef → Any
- def ilike(literal: String): Column
SQL ILIKE expression (case insensitive LIKE).
SQL ILIKE expression (case insensitive LIKE).
- Since
3.3.0
- def initializeLogIfNecessary(isInterpreter: Boolean, silent: Boolean): Boolean
- Attributes
- protected
- Definition Classes
- Logging
- def initializeLogIfNecessary(isInterpreter: Boolean): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def isInCollection(values: Iterable[_]): Column
A boolean expression that is evaluated to true if the value of this expression is contained by the provided collection.
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"
- Since
2.4.0
- def isInCollection(values: Iterable[_]): Column
A boolean expression that is evaluated to true if the value of this expression is contained by the provided collection.
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"
- Since
2.4.0
- final def isInstanceOf[T0]: Boolean
- Definition Classes
- Any
- def isNaN: Column
True if the current expression is NaN.
True if the current expression is NaN.
- Since
1.5.0
- def isNotNull: Column
True if the current expression is NOT null.
True if the current expression is NOT null.
- Since
1.3.0
- def isNull: Column
True if the current expression is null.
True if the current expression is null.
- Since
1.3.0
- def isTraceEnabled(): Boolean
- Attributes
- protected
- Definition Classes
- Logging
- def isin(list: Any*): Column
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.
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"
- Annotations
- @varargs()
- Since
1.5.0
- def leq(other: Any): Column
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.col("age").leq(21) );
- Since
1.3.0
- def like(literal: String): Column
SQL like expression.
SQL like expression. Returns a boolean column based on a SQL LIKE match.
- Since
1.3.0
- def log: Logger
- Attributes
- protected
- Definition Classes
- Logging
- def logDebug(msg: => String, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logDebug(entry: LogEntry, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logDebug(entry: LogEntry): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logDebug(msg: => String): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logError(msg: => String, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logError(entry: LogEntry, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logError(entry: LogEntry): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logError(msg: => String): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logInfo(msg: => String, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logInfo(entry: LogEntry, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logInfo(entry: LogEntry): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logInfo(msg: => String): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logName: String
- Attributes
- protected
- Definition Classes
- Logging
- def logTrace(msg: => String, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logTrace(entry: LogEntry, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logTrace(entry: LogEntry): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logTrace(msg: => String): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logWarning(msg: => String, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logWarning(entry: LogEntry, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logWarning(entry: LogEntry): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def logWarning(msg: => String): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def lt(other: Any): Column
Less than.
Less than.
// Scala: The following selects people younger than 21. people.select( people("age") < 21 ) // Java: people.select( people.col("age").lt(21) );
- Since
1.3.0
- def minus(other: Any): Column
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.col("height").minus(people.col("weight")) );
- Since
1.3.0
- def mod(other: Any): Column
Modulo (a.k.a.
Modulo (a.k.a. remainder) expression.
- Since
1.3.0
- def multiply(other: Any): Column
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.col("height").multiply(people.col("weight")) );
- Since
1.3.0
- def name(alias: String): Column
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 the API
as(alias: String, metadata: Metadata)
with explicit metadata.- Since
2.0.0
- final def ne(arg0: AnyRef): Boolean
- Definition Classes
- AnyRef
- def notEqual(other: Any): Column
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
- final def notify(): Unit
- Definition Classes
- AnyRef
- Annotations
- @IntrinsicCandidate() @native()
- final def notifyAll(): Unit
- Definition Classes
- AnyRef
- Annotations
- @IntrinsicCandidate() @native()
- def or(other: Column): Column
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.col("inSchool").or(people.col("isEmployed")) );
- Since
1.3.0
- def otherwise(value: Any): Column
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
- def over(): Column
Defines an empty analytic clause.
Defines an 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
- def over(window: WindowSpec): Column
Defines a windowing column.
Defines a windowing column.
val w = Window.partitionBy("name").orderBy("id") df.select( sum("price").over(w.rangeBetween(Window.unboundedPreceding, 2)), avg("price").over(w.rowsBetween(Window.currentRow, 4)) )
- Since
1.4.0
- def plus(other: Any): Column
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.col("height").plus(people.col("weight")) );
- Since
1.3.0
- def rlike(literal: String): Column
SQL RLIKE expression (LIKE with Regex).
SQL RLIKE expression (LIKE with Regex). Returns a boolean column based on a regex match.
- Since
1.3.0
- def startsWith(literal: String): Column
String starts with another string literal.
String starts with another string literal. Returns a boolean column based on a string match.
- Since
1.3.0
- def startsWith(other: Column): Column
String starts with.
String starts with. Returns a boolean column based on a string match.
- Since
1.3.0
- def substr(startPos: Int, len: Int): Column
An expression that returns a substring.
An expression that returns a substring.
- startPos
starting position.
- len
length of the substring.
- Since
1.3.0
- def substr(startPos: Column, len: Column): Column
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
- final def synchronized[T0](arg0: => T0): T0
- Definition Classes
- AnyRef
- def toString(): String
- Definition Classes
- Column → AnyRef → Any
- def try_cast(to: String): Column
Casts the column to a different data type and the result is null on failure.
Casts the column to a different data type and the result is null on failure.
// Casts colA to integer. df.select(df("colA").try_cast("int"))
- Since
4.0.0
- def try_cast(to: DataType): Column
Casts the column to a different data type and the result is null on failure.
Casts the column to a different data type and the result is null on failure.
// Casts colA to IntegerType. import org.apache.spark.sql.types.IntegerType df.select(df("colA").try_cast(IntegerType)) // equivalent to df.select(df("colA").try_cast("int"))
- Since
4.0.0
- def unary_!: Column
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
- def unary_-: Column
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
- final def wait(arg0: Long, arg1: Int): Unit
- Definition Classes
- AnyRef
- Annotations
- @throws(classOf[java.lang.InterruptedException])
- final def wait(arg0: Long): Unit
- Definition Classes
- AnyRef
- Annotations
- @throws(classOf[java.lang.InterruptedException]) @native()
- final def wait(): Unit
- Definition Classes
- AnyRef
- Annotations
- @throws(classOf[java.lang.InterruptedException])
- def when(condition: Column, value: Any): Column
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
- def withField(fieldName: String, col: Column): Column
An expression that adds/replaces field in
StructType
by name.An expression that adds/replaces field in
StructType
by name.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} 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}}
- Since
3.1.0
- def withLogContext(context: HashMap[String, String])(body: => Unit): Unit
- Attributes
- protected
- Definition Classes
- Logging
- def ||(other: Any): Column
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.col("inSchool").or(people.col("isEmployed")) );
- Since
1.3.0
Deprecated Value Members
- def !==(other: Any): Column
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
- def finalize(): Unit
- Attributes
- protected[lang]
- Definition Classes
- AnyRef
- Annotations
- @throws(classOf[java.lang.Throwable]) @Deprecated
- Deprecated
(Since version 9)