Skip to content

Commit

Permalink
[SPARK-36263][SQL][PYTHON] Add Dataframe.observation to PySpark
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?
With SPARK-34806 we can now easily add an equivalent for `Dataset.observe(Observation, Column, Column*)` to PySpark's `DataFrame` API.

### Why are the changes needed?
This further aligns the Python DataFrame API with Scala Dataset API.

### Does this PR introduce _any_ user-facing change?
Yes, it adds the `Observation` class and the `DataFrame.observe` method.

### How was this patch tested?
Adds test `test_observe` to `pyspark.sql.test.test_dataframe`.

Closes #33484 from EnricoMi/branch-observation-python.

Authored-by: Enrico Minack <github@enrico.minack.dev>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
  • Loading branch information
EnricoMi authored and cloud-fan committed Jul 27, 2021
1 parent df98d5b commit f90eb6a
Show file tree
Hide file tree
Showing 9 changed files with 283 additions and 2 deletions.
1 change: 1 addition & 0 deletions dev/sparktestsupport/modules.py
Original file line number Diff line number Diff line change
Expand Up @@ -437,6 +437,7 @@ def __hash__(self):
"pyspark.sql.pandas.serializers",
"pyspark.sql.pandas.typehints",
"pyspark.sql.pandas.utils",
"pyspark.sql.observation",
# unittests
"pyspark.sql.tests.test_arrow",
"pyspark.sql.tests.test_catalog",
Expand Down
13 changes: 13 additions & 0 deletions python/docs/source/reference/pyspark.sql.rst
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ Core Classes
Catalog
DataFrame
Column
Observation
Row
GroupedData
PandasCogroupedOps
Expand Down Expand Up @@ -177,6 +178,7 @@ DataFrame APIs
DataFrame.localCheckpoint
DataFrame.mapInPandas
DataFrame.na
DataFrame.observe
DataFrame.orderBy
DataFrame.persist
DataFrame.printSchema
Expand Down Expand Up @@ -299,6 +301,17 @@ Data Types
TimestampType


Observation
-----------

.. currentmodule:: pyspark.sql

.. autosummary::
:toctree: api/

Observation.get


Row
---

Expand Down
3 changes: 2 additions & 1 deletion python/pyspark/sql/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -46,14 +46,15 @@
from pyspark.sql.catalog import Catalog
from pyspark.sql.dataframe import DataFrame, DataFrameNaFunctions, DataFrameStatFunctions
from pyspark.sql.group import GroupedData
from pyspark.sql.observation import Observation
from pyspark.sql.readwriter import DataFrameReader, DataFrameWriter
from pyspark.sql.window import Window, WindowSpec
from pyspark.sql.pandas.group_ops import PandasCogroupedOps


__all__ = [
'SparkSession', 'SQLContext', 'HiveContext', 'UDFRegistration',
'DataFrame', 'GroupedData', 'Column', 'Catalog', 'Row',
'DataFrame', 'GroupedData', 'Column', 'Catalog', 'Observation', 'Row',
'DataFrameNaFunctions', 'DataFrameStatFunctions', 'Window', 'WindowSpec',
'DataFrameReader', 'DataFrameWriter', 'PandasCogroupedOps'
]
1 change: 1 addition & 0 deletions python/pyspark/sql/__init__.pyi
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ from pyspark.sql.dataframe import ( # noqa: F401
DataFrameStatFunctions as DataFrameStatFunctions,
)
from pyspark.sql.group import GroupedData as GroupedData # noqa: F401
from pyspark.sql.observation import Observation # noqa: F401
from pyspark.sql.pandas.group_ops import ( # noqa: F401
PandasCogroupedOps as PandasCogroupedOps,
)
Expand Down
41 changes: 40 additions & 1 deletion python/pyspark/sql/dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -301,7 +301,7 @@ def explain(self, extended=None, mode=None):
.. versionadded:: 1.3.0
parameters
Parameters
----------
extended : bool, optional
default ``False``. If ``False``, prints only the physical plan.
Expand Down Expand Up @@ -1830,6 +1830,45 @@ def agg(self, *exprs):
"""
return self.groupBy().agg(*exprs)

@since(3.3)
def observe(self, observation, *exprs):
"""Observe (named) metrics through an :class:`Observation` instance.
A user can retrieve the metrics by accessing `Observation.get`.
.. versionadded:: 3.3.0
Parameters
----------
observation : :class:`Observation`
an :class:`Observation` instance to obtain the metric.
exprs : list of :class:`Column`
column expressions (:class:`Column`).
Returns
-------
:class:`DataFrame`
the observed :class:`DataFrame`.
Notes
-----
This method does not support streaming datasets.
Examples
--------
>>> from pyspark.sql.functions import col, count, lit, max
>>> from pyspark.sql import Observation
>>> observation = Observation("my metrics")
>>> observed_df = df.observe(observation, count(lit(1)).alias("count"), max(col("age")))
>>> observed_df.count()
2
>>> observation.get
Row(count=2, max(age)=5)
"""
from pyspark.sql import Observation
assert isinstance(observation, Observation), "observation should be Observation"
return observation._on(self, *exprs)

@since(2.0)
def union(self, other):
""" Return a new :class:`DataFrame` containing union of rows in this and another
Expand Down
2 changes: 2 additions & 0 deletions python/pyspark/sql/dataframe.pyi
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ from pyspark.sql.types import ( # noqa: F401
) # noqa: F401
from pyspark.sql.context import SQLContext
from pyspark.sql.group import GroupedData
from pyspark.sql.observation import Observation
from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2
from pyspark.sql.streaming import DataStreamWriter
from pyspark.sql.column import Column
Expand Down Expand Up @@ -188,6 +189,7 @@ class DataFrame(PandasMapOpsMixin, PandasConversionMixin):
@overload
def cube(self, __cols: Union[List[Column], List[str]]) -> GroupedData: ...
def agg(self, *exprs: Union[Column, Dict[str, str]]) -> DataFrame: ...
def observe(self, observation: Observation, *exprs: Column) -> DataFrame: ...
def union(self, other: DataFrame) -> DataFrame: ...
def unionAll(self, other: DataFrame) -> DataFrame: ...
def unionByName(
Expand Down
146 changes: 146 additions & 0 deletions python/pyspark/sql/observation.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,146 @@
#
# 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.
#
from pyspark.sql import column, Column, DataFrame, Row

__all__ = ["Observation"]


class Observation:
"""Class to observe (named) metrics on a :class:`DataFrame`.
Metrics are aggregation expressions, which are applied to the DataFrame while is is being
processed by an action.
The metrics have the following guarantees:
- It will compute the defined aggregates (metrics) on all the data that is flowing through
the Dataset during the action.
- It will report the value of the defined aggregate columns as soon as we reach the end of
the action.
The metrics columns must either contain a literal (e.g. lit(42)), or should contain one or
more aggregate functions (e.g. sum(a) or sum(a + b) + avg(c) - lit(1)). Expressions that
contain references to the input Dataset's columns must always be wrapped in an aggregate
function.
An Observation instance collects the metrics while the first action is executed. Subsequent
actions do not modify the metrics returned by `Observation.get`. Retrieval of the metric via
`Observation.get` blocks until the first action has finished and metrics become available.
.. versionadded:: 3.3.0
Notes
-----
This class does not support streaming datasets.
Examples
--------
>>> from pyspark.sql.functions import col, count, lit, max
>>> from pyspark.sql import Observation
>>> df = spark.createDataFrame([["Alice", 2], ["Bob", 5]], ["name", "age"])
>>> observation = Observation("my metrics")
>>> observed_df = df.observe(observation, count(lit(1)).alias("count"), max(col("age")))
>>> observed_df.count()
2
>>> observation.get
Row(count=2, max(age)=5)
"""
def __init__(self, name=None):
"""Constructs a named or unnamed Observation instance.
Parameters
----------
name : str, optional
default is a random UUID string. This is the name of the Observation and the metric.
"""
if name is not None:
if not isinstance(name, str):
raise TypeError("name should be a string")
if name == '':
raise ValueError("name should not be empty")
self._name = name
self._jvm = None
self._jo = None

def _on(self, df, *exprs):
"""Attaches this observation to the given :class:`DataFrame` to observe aggregations.
Parameters
----------
df : :class:`DataFrame`
the :class:`DataFrame` to be observed
exprs : list of :class:`Column`
column expressions (:class:`Column`).
Returns
-------
:class:`DataFrame`
the observed :class:`DataFrame`.
"""
assert exprs, "exprs should not be empty"
assert all(isinstance(c, Column) for c in exprs), "all exprs should be Column"
assert self._jo is None, "an Observation can be used with a DataFrame only once"

self._jvm = df._sc._jvm
cls = self._jvm.org.apache.spark.sql.Observation
self._jo = cls(self._name) if self._name is not None else cls()
observed_df = self._jo.on(df._jdf,
exprs[0]._jc,
column._to_seq(df._sc, [c._jc for c in exprs[1:]]))
return DataFrame(observed_df, df.sql_ctx)

@property
def get(self):
"""Get the observed metrics.
Waits until the observed dataset finishes its first action. Only the result of the
first action is available. Subsequent actions do not modify the result.
Returns
-------
:class:`Row`
the observed metrics
"""
assert self._jo is not None, 'call DataFrame.observe'
jrow = self._jo.get()
return self._to_row(jrow)

def _to_row(self, jrow):
field_names = jrow.schema().fieldNames()
values_scala_map = jrow.getValuesMap(self._jvm.PythonUtils.toSeq(list(field_names)))
values_java_map = self._jvm.scala.collection.JavaConversions.mapAsJavaMap(values_scala_map)
return Row(**values_java_map)


def _test():
import doctest
import sys
from pyspark.context import SparkContext
from pyspark.sql import SparkSession
import pyspark.sql.observation
globs = pyspark.sql.observation.__dict__.copy()
sc = SparkContext('local[4]', 'PythonTest')
globs['spark'] = SparkSession(sc)

(failure_count, test_count) = doctest.testmod(pyspark.sql.observation, globs=globs)
sc.stop()
if failure_count:
sys.exit(-1)


if __name__ == "__main__":
_test()
29 changes: 29 additions & 0 deletions python/pyspark/sql/observation.pyi
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
#
# 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.

from typing import Optional

from py4j.java_gateway import JavaObject # type: ignore[import]

from pyspark.sql import Row


class Observation:
def __init__(self, name: Optional[str] = ...): ...
@property
def get(self) -> Row: ...
49 changes: 49 additions & 0 deletions python/pyspark/sql/tests/test_dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import unittest

from pyspark.sql import SparkSession, Row
from pyspark.sql.functions import col, lit, count, sum, mean
from pyspark.sql.types import StringType, IntegerType, DoubleType, StructType, StructField, \
BooleanType, DateType, TimestampType, FloatType
from pyspark.sql.utils import AnalysisException, IllegalArgumentException
Expand Down Expand Up @@ -389,6 +390,54 @@ def test_extended_hint_types(self):
self.assertEqual(1, logical_plan.toString().count("what"))
self.assertEqual(3, logical_plan.toString().count("itworks"))

def test_observe(self):
# SPARK-36263: tests the DataFrame.observe(Observation, *Column) method
from pyspark.sql import Observation

df = SparkSession(self.sc).createDataFrame([
(1, 1.0, 'one'),
(2, 2.0, 'two'),
(3, 3.0, 'three'),
], ['id', 'val', 'label'])

unnamed_observation = Observation()
named_observation = Observation("metric")
observed = df.orderBy('id').observe(
named_observation,
count(lit(1)).alias('cnt'),
sum(col("id")).alias('sum'),
mean(col("val")).alias('mean')
).observe(unnamed_observation, count(lit(1)).alias('rows'))

# test that observe works transparently
actual = observed.collect()
self.assertEqual([
{'id': 1, 'val': 1.0, 'label': 'one'},
{'id': 2, 'val': 2.0, 'label': 'two'},
{'id': 3, 'val': 3.0, 'label': 'three'},
], [row.asDict() for row in actual])

# test that we retrieve the metrics
self.assertEqual(named_observation.get, Row(cnt=3, sum=6, mean=2.0))
self.assertEqual(unnamed_observation.get, Row(rows=3))

# observation requires name (if given) to be non empty string
with self.assertRaisesRegex(TypeError, 'name should be a string'):
Observation(123)
with self.assertRaisesRegex(ValueError, 'name should not be empty'):
Observation('')

# dataframe.observe requires at least one expr
with self.assertRaisesRegex(AssertionError, 'exprs should not be empty'):
df.observe(Observation())

# dataframe.observe requires non-None Columns
for args in [(None,), ('id',),
(lit(1), None), (lit(1), 'id')]:
with self.subTest(args=args):
with self.assertRaisesRegex(AssertionError, 'all exprs should be Column'):
df.observe(Observation(), *args)

def test_sample(self):
self.assertRaisesRegex(
TypeError,
Expand Down

0 comments on commit f90eb6a

Please sign in to comment.