Hot-keys on this page
r m x p toggle line displays
j k next/prev highlighted chunk
0 (zero) top of page
1 (one) first highlighted chunk
# # 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. #
else: "Invalid argument, not a string or column: " "{0} of type {1}. " "For column literals, use 'lit', 'array', 'struct' or 'create_map' " "function.".format(col, type(col)))
""" Convert a list of Column (or names) into a JVM Seq of Column.
An optional `converter` could be used to convert items in `cols` into JVM Column objects. """
""" Convert a list of Column (or names) into a JVM (Scala) List of Column.
An optional `converter` could be used to convert items in `cols` into JVM Column objects. """ cols = [converter(c) for c in cols]
""" Create a method for given unary operator """
""" Create a method for given binary operator """
""" Create a method for binary operator (this object is on right side) """
""" A column in a DataFrame.
:class:`Column` instances can be created by::
# 1. Select a column out of a DataFrame
df.colName df["colName"]
# 2. Create from an expression df.colName + 1 1 / df.colName
.. versionadded:: 1.3.0 """
# arithmetic operators
# logistic operators
Equality test that is safe for null values.
.. versionadded:: 2.3.0
Parameters ---------- other a value or :class:`Column`
Examples -------- >>> from pyspark.sql import Row >>> df1 = spark.createDataFrame([ ... Row(id=1, value='foo'), ... Row(id=2, value=None) ... ]) >>> df1.select( ... df1['value'] == 'foo', ... df1['value'].eqNullSafe('foo'), ... df1['value'].eqNullSafe(None) ... ).show() +-------------+---------------+----------------+ |(value = foo)|(value <=> foo)|(value <=> NULL)| +-------------+---------------+----------------+ | true| true| false| | null| false| true| +-------------+---------------+----------------+ >>> df2 = spark.createDataFrame([ ... Row(value = 'bar'), ... Row(value = None) ... ]) >>> df1.join(df2, df1["value"] == df2["value"]).count() 0 >>> df1.join(df2, df1["value"].eqNullSafe(df2["value"])).count() 1 >>> df2 = spark.createDataFrame([ ... Row(id=1, value=float('NaN')), ... Row(id=2, value=42.0), ... Row(id=3, value=None) ... ]) >>> df2.select( ... df2['value'].eqNullSafe(None), ... df2['value'].eqNullSafe(float('NaN')), ... df2['value'].eqNullSafe(42.0) ... ).show() +----------------+---------------+----------------+ |(value <=> NULL)|(value <=> NaN)|(value <=> 42.0)| +----------------+---------------+----------------+ | false| true| false| | false| false| true| | true| false| false| +----------------+---------------+----------------+
Notes ----- Unlike Pandas, PySpark doesn't consider NaN values to be NULL. See the `NaN Semantics <https://spark.apache.org/docs/latest/sql-ref-datatypes.html#nan-semantics>`_ for details. """
# `and`, `or`, `not` cannot be overloaded in Python, # so use bitwise operators as boolean operators
# container operators "in a string column or 'array_contains' function for an array column.")
# bitwise operators Compute bitwise OR of this expression with another expression.
Parameters ---------- other a value or :class:`Column` to calculate bitwise or(|) with this :class:`Column`.
Examples -------- >>> from pyspark.sql import Row >>> df = spark.createDataFrame([Row(a=170, b=75)]) >>> df.select(df.a.bitwiseOR(df.b)).collect() [Row((a | b)=235)] """ Compute bitwise AND of this expression with another expression.
Parameters ---------- other a value or :class:`Column` to calculate bitwise and(&) with this :class:`Column`.
Examples -------- >>> from pyspark.sql import Row >>> df = spark.createDataFrame([Row(a=170, b=75)]) >>> df.select(df.a.bitwiseAND(df.b)).collect() [Row((a & b)=10)] """ Compute bitwise XOR of this expression with another expression.
Parameters ---------- other a value or :class:`Column` to calculate bitwise xor(^) with this :class:`Column`.
Examples -------- >>> from pyspark.sql import Row >>> df = spark.createDataFrame([Row(a=170, b=75)]) >>> df.select(df.a.bitwiseXOR(df.b)).collect() [Row((a ^ b)=225)] """
""" An expression that gets an item at position ``ordinal`` out of a list, or gets an item by key out of a dict.
.. versionadded:: 1.3.0
Examples -------- >>> df = spark.createDataFrame([([1, 2], {"key": "value"})], ["l", "d"]) >>> df.select(df.l.getItem(0), df.d.getItem("key")).show() +----+------+ |l[0]|d[key]| +----+------+ | 1| value| +----+------+ """ warnings.warn( "A column as 'key' in getItem is deprecated as of Spark 3.0, and will not " "be supported in the future release. Use `column[key]` or `column.key` syntax " "instead.", FutureWarning )
""" An expression that gets a field by name in a :class:`StructType`.
.. versionadded:: 1.3.0
Examples -------- >>> from pyspark.sql import Row >>> df = spark.createDataFrame([Row(r=Row(a=1, b="b"))]) >>> df.select(df.r.getField("b")).show() +---+ |r.b| +---+ | b| +---+ >>> df.select(df.r.a).show() +---+ |r.a| +---+ | 1| +---+ """ warnings.warn( "A column as 'name' in getField is deprecated as of Spark 3.0, and will not " "be supported in the future release. Use `column[name]` or `column.name` syntax " "instead.", FutureWarning )
""" An expression that adds/replaces a field in :class:`StructType` by name.
.. versionadded:: 3.1.0
Examples -------- >>> from pyspark.sql import Row >>> from pyspark.sql.functions import lit >>> df = spark.createDataFrame([Row(a=Row(b=1, c=2))]) >>> df.withColumn('a', df['a'].withField('b', lit(3))).select('a.b').show() +---+ | b| +---+ | 3| +---+ >>> df.withColumn('a', df['a'].withField('d', lit(4))).select('a.d').show() +---+ | d| +---+ | 4| +---+ """
""" An expression that drops fields in :class:`StructType` by name. This is a no-op if schema doesn't contain field name(s).
.. versionadded:: 3.1.0
Examples -------- >>> from pyspark.sql import Row >>> from pyspark.sql.functions import col, lit >>> df = spark.createDataFrame([ ... Row(a=Row(b=1, c=2, d=3, e=Row(f=4, g=5, h=6)))]) >>> df.withColumn('a', df['a'].dropFields('b')).show() +-----------------+ | a| +-----------------+ |{2, 3, {4, 5, 6}}| +-----------------+
>>> df.withColumn('a', df['a'].dropFields('b', 'c')).show() +--------------+ | a| +--------------+ |{3, {4, 5, 6}}| +--------------+
This method supports dropping multiple nested fields directly e.g.
>>> df.withColumn("a", col("a").dropFields("e.g", "e.h")).show() +--------------+ | a| +--------------+ |{1, 2, 3, {4}}| +--------------+
However, if you are going to add/replace multiple nested fields, it is preferred to extract out the nested struct before adding/replacing multiple fields e.g.
>>> df.select(col("a").withField( ... "e", col("a.e").dropFields("g", "h")).alias("a") ... ).show() +--------------+ | a| +--------------+ |{1, 2, 3, {4}}| +--------------+
"""
else:
# string methods Contains the other element. Returns a boolean :class:`Column` based on a string match.
Parameters ---------- other string in line. A value as a literal or a :class:`Column`.
Examples -------- >>> df.filter(df.name.contains('o')).collect() [Row(age=5, name='Bob')] """ SQL RLIKE expression (LIKE with Regex). Returns a boolean :class:`Column` based on a regex match.
Parameters ---------- other : str an extended regex expression
Examples -------- >>> df.filter(df.name.rlike('ice$')).collect() [Row(age=2, name='Alice')] """ SQL like expression. Returns a boolean :class:`Column` based on a SQL LIKE match.
Parameters ---------- other : str a SQL LIKE pattern
See Also -------- pyspark.sql.Column.rlike
Examples -------- >>> df.filter(df.name.like('Al%')).collect() [Row(age=2, name='Alice')] """ String starts with. Returns a boolean :class:`Column` based on a string match.
Parameters ---------- other : :class:`Column` or str string at start of line (do not use a regex `^`)
Examples -------- >>> df.filter(df.name.startswith('Al')).collect() [Row(age=2, name='Alice')] >>> df.filter(df.name.startswith('^Al')).collect() [] """ String ends with. Returns a boolean :class:`Column` based on a string match.
Parameters ---------- other : :class:`Column` or str string at end of line (do not use a regex `$`)
Examples -------- >>> df.filter(df.name.endswith('ice')).collect() [Row(age=2, name='Alice')] >>> df.filter(df.name.endswith('ice$')).collect() [] """
""" Return a :class:`Column` which is a substring of the column.
.. versionadded:: 1.3.0
Parameters ---------- startPos : :class:`Column` or int start position length : :class:`Column` or int length of the substring
Examples -------- >>> df.select(df.name.substr(1, 3).alias("col")).collect() [Row(col='Ali'), Row(col='Bob')] """ "startPos and length must be the same type. " "Got {startPos_t} and {length_t}, respectively." .format( startPos_t=type(startPos), length_t=type(length), )) elif isinstance(startPos, Column): jc = self._jc.substr(startPos._jc, length._jc) else: raise TypeError("Unexpected type: %s" % type(startPos))
""" A boolean expression that is evaluated to true if the value of this expression is contained by the evaluated values of the arguments.
.. versionadded:: 1.5.0
Examples -------- >>> df[df.name.isin("Bob", "Mike")].collect() [Row(age=5, name='Bob')] >>> df[df.age.isin([1, 2, 3])].collect() [Row(age=2, name='Alice')] """
# order Returns a sort expression based on ascending order of the column.
Examples -------- >>> from pyspark.sql import Row >>> df = spark.createDataFrame([('Tom', 80), ('Alice', None)], ["name", "height"]) >>> df.select(df.name).orderBy(df.name.asc()).collect() [Row(name='Alice'), Row(name='Tom')] """ Returns a sort expression based on ascending order of the column, and null values return before non-null values.
.. versionadded:: 2.4.0
Examples -------- >>> from pyspark.sql import Row >>> df = spark.createDataFrame([('Tom', 80), (None, 60), ('Alice', None)], ["name", "height"]) >>> df.select(df.name).orderBy(df.name.asc_nulls_first()).collect() [Row(name=None), Row(name='Alice'), Row(name='Tom')]
""" Returns a sort expression based on ascending order of the column, and null values appear after non-null values.
.. versionadded:: 2.4.0
Examples -------- >>> from pyspark.sql import Row >>> df = spark.createDataFrame([('Tom', 80), (None, 60), ('Alice', None)], ["name", "height"]) >>> df.select(df.name).orderBy(df.name.asc_nulls_last()).collect() [Row(name='Alice'), Row(name='Tom'), Row(name=None)]
""" Returns a sort expression based on the descending order of the column.
.. versionadded:: 2.4.0
Examples -------- >>> from pyspark.sql import Row >>> df = spark.createDataFrame([('Tom', 80), ('Alice', None)], ["name", "height"]) >>> df.select(df.name).orderBy(df.name.desc()).collect() [Row(name='Tom'), Row(name='Alice')] """ Returns a sort expression based on the descending order of the column, and null values appear before non-null values.
.. versionadded:: 2.4.0
Examples -------- >>> from pyspark.sql import Row >>> df = spark.createDataFrame([('Tom', 80), (None, 60), ('Alice', None)], ["name", "height"]) >>> df.select(df.name).orderBy(df.name.desc_nulls_first()).collect() [Row(name=None), Row(name='Tom'), Row(name='Alice')]
""" Returns a sort expression based on the descending order of the column, and null values appear after non-null values.
.. versionadded:: 2.4.0
Examples -------- >>> from pyspark.sql import Row >>> df = spark.createDataFrame([('Tom', 80), (None, 60), ('Alice', None)], ["name", "height"]) >>> df.select(df.name).orderBy(df.name.desc_nulls_last()).collect() [Row(name='Tom'), Row(name='Alice'), Row(name=None)] """
True if the current expression is null.
Examples -------- >>> from pyspark.sql import Row >>> df = spark.createDataFrame([Row(name='Tom', height=80), Row(name='Alice', height=None)]) >>> df.filter(df.height.isNull()).collect() [Row(name='Alice', height=None)] """ True if the current expression is NOT null.
Examples -------- >>> from pyspark.sql import Row >>> df = spark.createDataFrame([Row(name='Tom', height=80), Row(name='Alice', height=None)]) >>> df.filter(df.height.isNotNull()).collect() [Row(name='Tom', height=80)] """
""" Returns this column aliased with a new name or names (in the case of expressions that return more than one column, such as explode).
.. versionadded:: 1.3.0
Parameters ---------- alias : str desired column names (collects all positional arguments passed)
Other Parameters ---------------- metadata: dict a dict of information to be stored in ``metadata`` attribute of the corresponding :class:`StructField <pyspark.sql.types.StructField>` (optional, keyword only argument)
.. versionchanged:: 2.2.0 Added optional ``metadata`` argument.
Examples -------- >>> df.select(df.age.alias("age2")).collect() [Row(age2=2), Row(age2=5)] >>> df.select(df.age.alias("age3", metadata={'max': 99})).schema['age3'].metadata['max'] 99 """
json.dumps(metadata)) else: else: raise ValueError('metadata can only be provided for a single column')
""" Casts the column into type ``dataType``.
.. versionadded:: 1.3.0
Examples -------- >>> df.select(df.age.cast("string").alias('ages')).collect() [Row(ages='2'), Row(ages='5')] >>> df.select(df.age.cast(StringType()).alias('ages')).collect() [Row(ages='2'), Row(ages='5')] """ else: raise TypeError("unexpected type: %s" % type(dataType))
""" True if the current column is between the lower bound and upper bound, inclusive.
.. versionadded:: 1.3.0
Examples -------- >>> df.select(df.name, df.age.between(2, 4)).show() +-----+---------------------------+ | name|((age >= 2) AND (age <= 4))| +-----+---------------------------+ |Alice| true| | Bob| false| +-----+---------------------------+ """
""" Evaluates a list of conditions and returns one of multiple possible result expressions. If :func:`Column.otherwise` is not invoked, None is returned for unmatched conditions.
.. versionadded:: 1.4.0
Parameters ---------- condition : :class:`Column` a boolean :class:`Column` expression. value a literal value, or a :class:`Column` expression.
Examples -------- >>> from pyspark.sql import functions as F >>> df.select(df.name, F.when(df.age > 4, 1).when(df.age < 3, -1).otherwise(0)).show() +-----+------------------------------------------------------------+ | name|CASE WHEN (age > 4) THEN 1 WHEN (age < 3) THEN -1 ELSE 0 END| +-----+------------------------------------------------------------+ |Alice| -1| | Bob| 1| +-----+------------------------------------------------------------+
See Also -------- pyspark.sql.functions.when """ raise TypeError("condition should be a Column")
""" Evaluates a list of conditions and returns one of multiple possible result expressions. If :func:`Column.otherwise` is not invoked, None is returned for unmatched conditions.
.. versionadded:: 1.4.0
Parameters ---------- value a literal value, or a :class:`Column` expression.
Examples -------- >>> from pyspark.sql import functions as F >>> df.select(df.name, F.when(df.age > 3, 1).otherwise(0)).show() +-----+-------------------------------------+ | name|CASE WHEN (age > 3) THEN 1 ELSE 0 END| +-----+-------------------------------------+ |Alice| 0| | Bob| 1| +-----+-------------------------------------+
See Also -------- pyspark.sql.functions.when """
""" Define a windowing column.
.. versionadded:: 1.4.0
Parameters ---------- window : :class:`WindowSpec`
Returns ------- :class:`Column`
Examples -------- >>> from pyspark.sql import Window >>> window = Window.partitionBy("name").orderBy("age") \ .rowsBetween(Window.unboundedPreceding, Window.currentRow) >>> from pyspark.sql.functions import rank, min >>> from pyspark.sql.functions import desc >>> df.withColumn("rank", rank().over(window)) \ .withColumn("min", min('age').over(window)).sort(desc("age")).show() +---+-----+----+---+ |age| name|rank|min| +---+-----+----+---+ | 5| Bob| 1| 5| | 2|Alice| 1| 2| +---+-----+----+---+ """ raise TypeError("window should be WindowSpec")
"'~' for 'not' when building DataFrame boolean expressions.")
.master("local[4]")\ .appName("sql.column tests")\ .getOrCreate() .toDF(StructType([StructField('age', IntegerType()), StructField('name', StringType())]))
pyspark.sql.column, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF) sys.exit(-1)
|