blob: 497fd73b0be683462d11a545c8f7ff9c2155aea1 [file] [log] [blame]
#
# 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.
#
import os
import time
import unittest
import logging
from pyspark.sql.utils import PythonException
from pyspark.testing.sqlutils import (
ReusedSQLTestCase,
have_pandas,
have_pyarrow,
pandas_requirement_message,
pyarrow_requirement_message,
)
from pyspark.sql import Row
from pyspark.testing.utils import assertDataFrameEqual
from pyspark.util import is_remote_only
if have_pyarrow:
import pyarrow as pa
if have_pandas:
import pandas as pd
@unittest.skipIf(
not have_pandas or not have_pyarrow,
pandas_requirement_message or pyarrow_requirement_message,
)
class MapInArrowTestsMixin(object):
def test_map_in_arrow(self):
def func(iterator):
for batch in iterator:
assert isinstance(batch, pa.RecordBatch)
assert batch.schema.names == ["id"]
yield batch
df = self.spark.range(10)
actual = df.mapInArrow(func, "id long").collect()
expected = df.collect()
self.assertEqual(actual, expected)
def test_multiple_columns(self):
data = [(1, "foo"), (2, None), (3, "bar"), (4, "bar")]
df = self.spark.createDataFrame(data, "a int, b string")
def func(iterator):
for batch in iterator:
assert isinstance(batch, pa.RecordBatch)
assert batch.schema.types == [pa.int32(), pa.string()]
yield batch
actual = df.mapInArrow(func, df.schema).collect()
expected = df.collect()
self.assertEqual(actual, expected)
def test_large_variable_width_types(self):
with self.sql_conf({"spark.sql.execution.arrow.useLargeVarTypes": True}):
data = [("foo", b"foo"), (None, None), ("bar", b"bar")]
df = self.spark.createDataFrame(data, "a string, b binary")
def func(iterator):
for batch in iterator:
assert isinstance(batch, pa.RecordBatch)
assert batch.schema.types == [pa.large_string(), pa.large_binary()]
yield batch
actual = df.mapInArrow(func, df.schema).collect()
expected = df.collect()
self.assertEqual(actual, expected)
def test_different_output_length(self):
def func(iterator):
for _ in iterator:
yield pa.RecordBatch.from_pandas(pd.DataFrame({"a": list(range(100))}))
df = self.spark.range(10)
actual = df.repartition(1).mapInArrow(func, "a long").collect()
self.assertEqual(set((r.a for r in actual)), set(range(100)))
def test_other_than_recordbatch_iter(self):
with self.quiet():
self.check_other_than_recordbatch_iter()
def check_other_than_recordbatch_iter(self):
def not_iter(_):
return 1
def bad_iter_elem(_):
return iter([1])
with self.assertRaisesRegex(
PythonException,
"Return type of the user-defined function should be iterator "
"of pyarrow.RecordBatch, but is int",
):
(self.spark.range(10, numPartitions=3).mapInArrow(not_iter, "a int").count())
with self.assertRaisesRegex(
PythonException,
"Return type of the user-defined function should be iterator "
"of pyarrow.RecordBatch, but is iterator of int",
):
(self.spark.range(10, numPartitions=3).mapInArrow(bad_iter_elem, "a int").count())
def test_empty_iterator(self):
def empty_iter(_):
return iter([])
self.assertEqual(self.spark.range(10).mapInArrow(empty_iter, "a int, b string").count(), 0)
def test_empty_rows(self):
def empty_rows(_):
return iter([pa.RecordBatch.from_pandas(pd.DataFrame({"a": []}))])
self.assertEqual(self.spark.range(10).mapInArrow(empty_rows, "a double").count(), 0)
def test_chain_map_in_arrow(self):
def func(iterator):
for batch in iterator:
assert isinstance(batch, pa.RecordBatch)
assert batch.schema.names == ["id"]
yield batch
df = self.spark.range(10)
actual = df.mapInArrow(func, "id long").mapInArrow(func, "id long").collect()
expected = df.collect()
self.assertEqual(actual, expected)
def test_self_join(self):
df1 = self.spark.range(10)
df2 = df1.mapInArrow(lambda iter: iter, "id long")
actual = df2.join(df2).collect()
expected = df1.join(df1).collect()
self.assertEqual(sorted(actual), sorted(expected))
def test_map_in_arrow_with_barrier_mode(self):
df = self.spark.range(10)
def func1(iterator):
from pyspark import TaskContext, BarrierTaskContext
tc = TaskContext.get()
assert tc is not None
assert not isinstance(tc, BarrierTaskContext)
for batch in iterator:
yield batch
df.mapInArrow(func1, "id long", False).collect()
def func2(iterator):
from pyspark import TaskContext, BarrierTaskContext
tc = TaskContext.get()
assert tc is not None
assert isinstance(tc, BarrierTaskContext)
for batch in iterator:
yield batch
df.mapInArrow(func2, "id long", True).collect()
def test_negative_and_zero_batch_size(self):
for batch_size in [0, -1]:
with self.sql_conf({"spark.sql.execution.arrow.maxRecordsPerBatch": batch_size}):
MapInArrowTests.test_map_in_arrow(self)
def test_nested_extraneous_field(self):
with self.sql_conf({"spark.sql.execution.arrow.pyspark.validateSchema.enabled": True}):
def func(iterator):
for _ in iterator:
struct_arr = pa.StructArray.from_arrays([[1, 2], [3, 4]], names=["a", "b"])
yield pa.RecordBatch.from_arrays([struct_arr], ["x"])
df = self.spark.range(1)
with self.assertRaisesRegex(Exception, r"ARROW_TYPE_MISMATCH.*SQL_MAP_ARROW_ITER_UDF"):
df.mapInArrow(func, "x struct<b:int>").collect()
def test_top_level_wrong_order(self):
with self.sql_conf({"spark.sql.execution.arrow.pyspark.validateSchema.enabled": True}):
def func(iterator):
for _ in iterator:
yield pa.RecordBatch.from_arrays([[1], [2]], ["b", "a"])
df = self.spark.range(1)
with self.assertRaisesRegex(Exception, r"ARROW_TYPE_MISMATCH.*SQL_MAP_ARROW_ITER_UDF"):
df.mapInArrow(func, "a int, b int").collect()
def test_nullability_widen(self):
with self.sql_conf({"spark.sql.execution.arrow.pyspark.validateSchema.enabled": True}):
def func(iterator):
for _ in iterator:
yield pa.RecordBatch.from_arrays([[1]], ["a"])
df = self.spark.range(1)
with self.assertRaisesRegex(Exception, r"ARROW_TYPE_MISMATCH.*SQL_MAP_ARROW_ITER_UDF"):
df.mapInArrow(func, "a int not null").collect()
def test_nullability_narrow(self):
with self.sql_conf({"spark.sql.execution.arrow.pyspark.validateSchema.enabled": True}):
def func(iterator):
for _ in iterator:
yield pa.RecordBatch.from_arrays(
[[1]], pa.schema([pa.field("a", pa.int32(), nullable=False)])
)
df = self.spark.range(1)
df.mapInArrow(func, "a int").collect()
@unittest.skipIf(is_remote_only(), "Requires JVM access")
def test_map_in_arrow_with_logging(self):
import pyarrow as pa
def func_with_logging(iterator):
logger = logging.getLogger("test_arrow_map")
for batch in iterator:
assert isinstance(batch, pa.RecordBatch)
logger.warning(f"arrow map: {batch.to_pydict()}")
yield batch
with self.sql_conf(
{
"spark.sql.execution.arrow.maxRecordsPerBatch": "3",
"spark.sql.pyspark.worker.logging.enabled": "true",
}
):
assertDataFrameEqual(
self.spark.range(9, numPartitions=2).mapInArrow(func_with_logging, "id long"),
[Row(id=i) for i in range(9)],
)
logs = self.spark.tvf.python_worker_logs()
assertDataFrameEqual(
logs.select("level", "msg", "context", "logger"),
self._expected_logs_for_test_map_in_arrow_with_logging(func_with_logging.__name__),
)
def _expected_logs_for_test_map_in_arrow_with_logging(self, func_name):
return [
Row(
level="WARNING",
msg=f"arrow map: {dict(id=lst)}",
context={"func_name": func_name},
logger="test_arrow_map",
)
for lst in [[0, 1, 2], [3], [4, 5, 6], [7, 8]]
]
class MapInArrowTests(MapInArrowTestsMixin, ReusedSQLTestCase):
@classmethod
def setUpClass(cls):
ReusedSQLTestCase.setUpClass()
# Synchronize default timezone between Python and Java
cls.tz_prev = os.environ.get("TZ", None) # save current tz if set
tz = "America/Los_Angeles"
os.environ["TZ"] = tz
time.tzset()
cls.sc.environment["TZ"] = tz
cls.spark.conf.set("spark.sql.session.timeZone", tz)
@classmethod
def tearDownClass(cls):
del os.environ["TZ"]
if cls.tz_prev is not None:
os.environ["TZ"] = cls.tz_prev
time.tzset()
ReusedSQLTestCase.tearDownClass()
class MapInArrowWithArrowBatchSlicingTestsAndReducedBatchSizeTests(MapInArrowTests):
@classmethod
def setUpClass(cls):
MapInArrowTests.setUpClass()
# Set it to a small odd value to exercise batching logic for all test cases
cls.spark.conf.set("spark.sql.execution.arrow.maxRecordsPerBatch", "3")
cls.spark.conf.set("spark.sql.execution.arrow.maxBytesPerBatch", "10")
def _expected_logs_for_test_map_in_arrow_with_logging(self, func_name):
return [
Row(
level="WARNING",
msg=f"arrow map: {dict(id=[i])}",
context={"func_name": func_name},
logger="test_arrow_map",
)
for i in range(9)
]
class MapInArrowWithOutputArrowBatchSlicingRecordsTests(MapInArrowTests):
@classmethod
def setUpClass(cls):
MapInArrowTests.setUpClass()
cls.spark.conf.set("spark.sql.execution.arrow.maxRecordsPerBatch", "10")
cls.spark.conf.set("spark.sql.execution.arrow.maxRecordsPerOutputBatch", "3")
class MapInArrowWithOutputArrowBatchSlicingBytesTests(MapInArrowTests):
@classmethod
def setUpClass(cls):
MapInArrowTests.setUpClass()
cls.spark.conf.set("spark.sql.execution.arrow.maxRecordsPerBatch", "10")
cls.spark.conf.set("spark.sql.execution.arrow.maxBytesPerOutputBatch", "3")
if __name__ == "__main__":
from pyspark.testing import main
main()