Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
22 commits
Select commit Hold shift + click to select a range
d1373fb
Support ANSI SQL intervals by the aggregate function `sum`
beliefer Apr 9, 2021
7897af4
Update sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregat…
HyukjinKwon Apr 11, 2021
75f3836
Add tests
beliefer Apr 13, 2021
a5bf107
Merge branch 'SPARK-34716' of github.com:beliefer/spark into SPARK-34716
beliefer Apr 13, 2021
9b8c461
Add tests
beliefer Apr 13, 2021
a4d1214
extend BufferSetterGetterUtils
beliefer Apr 13, 2021
464c4f0
test
beliefer Apr 14, 2021
faa928c
[MINOR][PYTHON][DOCS] Fix docstring for pyspark.sql.DataFrameWriter.j…
AlexMooney Apr 14, 2021
700aa17
[SPARK-35050][DOCS][MESOS] Document deprecation of Apache Mesos in 3.2.0
srowen Apr 14, 2021
cd1e8e8
[SPARK-35033][PYTHON] Port Koalas plot unit tests into PySpark
xinrong-meng Apr 14, 2021
2974b70
[SPARK-35048][INFRA] Distribute GitHub Actions workflows to fork repo…
HyukjinKwon Apr 14, 2021
47d62af
[SPARK-35035][PYTHON] Port Koalas internal implementation unit tests …
xinrong-meng Apr 14, 2021
4ae57d5
[SPARK-35039][PYTHON] Remove PySpark version dependent codes
ueshin Apr 14, 2021
0fc97b5
[SPARK-34577][SQL][FOLLOWUP] Add change of `DESC NAMESPACE`'s schema …
AngersZhuuuu Apr 14, 2021
a694b59
Support ANSI SQL intervals by the aggregate function `sum`
beliefer Apr 9, 2021
a9e1f7b
Update sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregat…
HyukjinKwon Apr 11, 2021
e779855
Add tests
beliefer Apr 13, 2021
0d11e37
Add tests
beliefer Apr 13, 2021
20766b1
extend BufferSetterGetterUtils
beliefer Apr 13, 2021
0a7bfc9
test
beliefer Apr 14, 2021
1176f1c
Update code
beliefer Apr 14, 2021
2624c27
Merge branch 'SPARK-34716-test' of github.com:beliefer/spark into SPA…
beliefer Apr 14, 2021
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
56 changes: 35 additions & 21 deletions .github/workflows/build_and_test.yml
Original file line number Diff line number Diff line change
Expand Up @@ -3,15 +3,8 @@ name: Build and test
on:
push:
branches:
- master
pull_request:
branches:
- master
workflow_dispatch:
inputs:
target:
description: 'Target branch to run'
required: true
- '**'
- '!branch-*.*'

jobs:
# Build: build Spark and run the tests for specified modules.
Expand Down Expand Up @@ -82,17 +75,23 @@ jobs:
# GitHub Actions' default miniconda to use in pip packaging test.
CONDA_PREFIX: /usr/share/miniconda
GITHUB_PREV_SHA: ${{ github.event.before }}
GITHUB_INPUT_BRANCH: ${{ github.event.inputs.target }}
SPARK_LOCAL_IP: localhost
steps:
- name: Checkout Spark repository
uses: actions/checkout@v2
# In order to fetch changed files
with:
fetch-depth: 0
- name: Merge dispatched input branch
if: ${{ github.event.inputs.target != '' }}
run: git merge --progress --ff-only origin/${{ github.event.inputs.target }}
repository: apache/spark
ref: master
- name: Sync the current branch with the latest in Apache Spark
if: github.repository != 'apache/spark'
id: sync-branch
run: |
apache_spark_ref=`git rev-parse HEAD`
git fetch https://github.com/$GITHUB_REPOSITORY.git ${GITHUB_REF##*/}
git merge --progress --ff-only FETCH_HEAD
echo "::set-output name=APACHE_SPARK_REF::$apache_spark_ref"
# Cache local repositories. Note that GitHub Actions cache has a 2G limit.
- name: Cache Scala, SBT and Maven
uses: actions/cache@v2
Expand Down Expand Up @@ -133,6 +132,7 @@ jobs:
# Run the tests.
- name: Run tests
run: |
export APACHE_SPARK_REF=${{ steps.sync-branch.outputs.APACHE_SPARK_REF }}
# Hive and SQL tests become flaky when running in parallel as it's too intensive.
if [[ "$MODULES_TO_TEST" == "hive" ]] || [[ "$MODULES_TO_TEST" == "sql" ]]; then export SERIAL_SBT_TESTS=1; fi
./dev/run-tests --parallelism 2 --modules "$MODULES_TO_TEST" --included-tags "$INCLUDED_TAGS" --excluded-tags "$EXCLUDED_TAGS"
Expand Down Expand Up @@ -171,17 +171,23 @@ jobs:
# GitHub Actions' default miniconda to use in pip packaging test.
CONDA_PREFIX: /usr/share/miniconda
GITHUB_PREV_SHA: ${{ github.event.before }}
GITHUB_INPUT_BRANCH: ${{ github.event.inputs.target }}
SPARK_LOCAL_IP: localhost
steps:
- name: Checkout Spark repository
uses: actions/checkout@v2
# In order to fetch changed files
with:
fetch-depth: 0
- name: Merge dispatched input branch
if: ${{ github.event.inputs.target != '' }}
run: git merge --progress --ff-only origin/${{ github.event.inputs.target }}
repository: apache/spark
ref: master
- name: Sync the current branch with the latest in Apache Spark
if: github.repository != 'apache/spark'
id: sync-branch
run: |
apache_spark_ref=`git rev-parse HEAD`
git fetch https://github.com/$GITHUB_REPOSITORY.git ${GITHUB_REF##*/}
git merge --progress --ff-only FETCH_HEAD
echo "::set-output name=APACHE_SPARK_REF::$apache_spark_ref"
# Cache local repositories. Note that GitHub Actions cache has a 2G limit.
- name: Cache Scala, SBT and Maven
uses: actions/cache@v2
Expand Down Expand Up @@ -216,6 +222,7 @@ jobs:
# Run the tests.
- name: Run tests
run: |
export APACHE_SPARK_REF=${{ steps.sync-branch.outputs.APACHE_SPARK_REF }}
./dev/run-tests --parallelism 2 --modules "$MODULES_TO_TEST"
- name: Upload test results to report
if: always()
Expand All @@ -239,17 +246,23 @@ jobs:
HADOOP_PROFILE: hadoop3.2
HIVE_PROFILE: hive2.3
GITHUB_PREV_SHA: ${{ github.event.before }}
GITHUB_INPUT_BRANCH: ${{ github.event.inputs.target }}
SPARK_LOCAL_IP: localhost
steps:
- name: Checkout Spark repository
uses: actions/checkout@v2
# In order to fetch changed files
with:
fetch-depth: 0
- name: Merge dispatched input branch
if: ${{ github.event.inputs.target != '' }}
run: git merge --progress --ff-only origin/${{ github.event.inputs.target }}
repository: apache/spark
ref: master
- name: Sync the current branch with the latest in Apache Spark
if: github.repository != 'apache/spark'
id: sync-branch
run: |
apache_spark_ref=`git rev-parse HEAD`
git fetch https://github.com/$GITHUB_REPOSITORY.git ${GITHUB_REF##*/}
git merge --progress --ff-only FETCH_HEAD
echo "::set-output name=APACHE_SPARK_REF::$apache_spark_ref"
# Cache local repositories. Note that GitHub Actions cache has a 2G limit.
- name: Cache Scala, SBT and Maven
uses: actions/cache@v2
Expand All @@ -275,6 +288,7 @@ jobs:
# R issues at docker environment
export TZ=UTC
export _R_CHECK_SYSTEM_CLOCK_=FALSE
export APACHE_SPARK_REF=${{ steps.sync-branch.outputs.APACHE_SPARK_REF }}
./dev/run-tests --parallelism 2 --modules sparkr
- name: Upload test results to report
if: always()
Expand Down
40 changes: 40 additions & 0 deletions .github/workflows/notify_test_workflow.yml
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
name: Notify test workflow
on:
pull_request_target:
types: [opened, reopened, synchronize]

jobs:
notify:
runs-on: ubuntu-20.04
steps:
- name: "Notify test workflow"
uses: actions/github-script@v3
if: ${{ github.base_ref == 'master' }}
with:
github-token: ${{ secrets.GITHUB_TOKEN }}
script: |
const endpoint = "GET /repos/:owner/:repo/actions/workflows/:id/runs?&branch=:branch"
const params = {
owner: context.payload.pull_request.head.repo.owner.login,
repo: context.payload.pull_request.head.repo.name,
id: "build_and_test.yml",
branch: context.payload.pull_request.head.ref,
}

const runs = await github.request(endpoint, params)
var runID = runs.data.workflow_runs[0].id

var msg = "**[Test build #" + runID + "]"
+ "(https://github.com/" + context.payload.pull_request.head.repo.full_name
+ "/actions/runs/" + runID + ")** "
+ "for PR " + context.issue.number
+ " at commit [`" + context.payload.pull_request.head.sha.substring(0, 7) + "`]"
+ "(https://github.com/" + context.payload.pull_request.head.repo.full_name
+ "/commit/" + context.payload.pull_request.head.sha + ")."

github.issues.createComment({
issue_number: context.issue.number,
owner: context.payload.repository.owner.login,
repo: context.payload.repository.name,
body: msg
})
12 changes: 3 additions & 9 deletions dev/run-tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -656,16 +656,10 @@ def main():
# If we're running the tests in GitHub Actions, attempt to detect and test
# only the affected modules.
if test_env == "github_actions":
if os.environ["GITHUB_INPUT_BRANCH"] != "":
# Dispatched request
# Note that it assumes GitHub Actions has already merged
# the given `GITHUB_INPUT_BRANCH` branch.
if os.environ["APACHE_SPARK_REF"] != "":
# Fork repository
changed_files = identify_changed_files_from_git_commits(
"HEAD", target_branch=os.environ["GITHUB_SHA"])
elif os.environ["GITHUB_BASE_REF"] != "":
# Pull requests
changed_files = identify_changed_files_from_git_commits(
os.environ["GITHUB_SHA"], target_branch=os.environ["GITHUB_BASE_REF"])
"HEAD", target_ref=os.environ["APACHE_SPARK_REF"])
else:
# Build for each commit.
changed_files = identify_changed_files_from_git_commits(
Expand Down
13 changes: 13 additions & 0 deletions dev/sparktestsupport/modules.py
Original file line number Diff line number Diff line change
Expand Up @@ -612,6 +612,13 @@ def __hash__(self):
"pyspark.pandas.typedef.typehints",
# unittests
"pyspark.pandas.tests.test_dataframe",
"pyspark.pandas.tests.test_config",
"pyspark.pandas.tests.test_default_index",
"pyspark.pandas.tests.test_extension",
"pyspark.pandas.tests.test_internal",
"pyspark.pandas.tests.test_numpy_compat",
"pyspark.pandas.tests.test_typedef",
"pyspark.pandas.tests.test_utils",
"pyspark.pandas.tests.test_dataframe_conversion",
"pyspark.pandas.tests.test_dataframe_spark_io",
"pyspark.pandas.tests.test_frame_spark",
Expand All @@ -623,6 +630,12 @@ def __hash__(self):
"pyspark.pandas.tests.test_series_conversion",
"pyspark.pandas.tests.test_series_datetime",
"pyspark.pandas.tests.test_series_string",
"pyspark.pandas.tests.plot.test_frame_plot",
"pyspark.pandas.tests.plot.test_frame_plot_matplotlib",
"pyspark.pandas.tests.plot.test_frame_plot_plotly",
"pyspark.pandas.tests.plot.test_series_plot",
"pyspark.pandas.tests.plot.test_series_plot_matplotlib",
"pyspark.pandas.tests.plot.test_series_plot_plotly",
],
excluded_python_implementations=[
"PyPy" # Skip these tests under PyPy since they require numpy, pandas, and pyarrow and
Expand Down
2 changes: 1 addition & 1 deletion docs/cluster-overview.md
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ The system currently supports several cluster managers:
* [Standalone](spark-standalone.html) -- a simple cluster manager included with Spark that makes it
easy to set up a cluster.
* [Apache Mesos](running-on-mesos.html) -- a general cluster manager that can also run Hadoop MapReduce
and service applications.
and service applications. (Deprecated)
* [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2.
* [Kubernetes](running-on-kubernetes.html) -- an open-source system for automating deployment, scaling,
and management of containerized applications.
Expand Down
2 changes: 2 additions & 0 deletions docs/core-migration-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,8 @@ license: |

- In Spark 3.2, `spark.launcher.childConectionTimeout` is deprecated (typo) though still works. Use `spark.launcher.childConnectionTimeout` instead.

- In Spark 3.2, support for Apache Mesos as a resource manager is deprecated and will be removed in a future version.

## Upgrading from Core 3.0 to 3.1

- In Spark 3.0 and below, `SparkContext` can be created in executors. Since Spark 3.1, an exception will be thrown when creating `SparkContext` in executors. You can allow it by setting the configuration `spark.executor.allowSparkContext` when creating `SparkContext` in executors.
Expand Down
2 changes: 1 addition & 1 deletion docs/index.md
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ Spark can run both by itself, or over several existing cluster managers. It curr
options for deployment:

* [Standalone Deploy Mode](spark-standalone.html): simplest way to deploy Spark on a private cluster
* [Apache Mesos](running-on-mesos.html)
* [Apache Mesos](running-on-mesos.html) (deprecated)
* [Hadoop YARN](running-on-yarn.html)
* [Kubernetes](running-on-kubernetes.html)

Expand Down
2 changes: 2 additions & 0 deletions docs/running-on-mesos.md
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@ license: |
---
* This will become a table of contents (this text will be scraped).
{:toc}

*Note*: Apache Mesos support is deprecated as of Apache Spark 3.2.0. It will be removed in a future version.

Spark can run on hardware clusters managed by [Apache Mesos](http://mesos.apache.org/).

Expand Down
6 changes: 4 additions & 2 deletions docs/sql-migration-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,10 @@ license: |

- In Spark 3.2, the output schema of `SHOW TABLE EXTENDED` becomes `namespace: string, tableName: string, isTemporary: boolean, information: string`. In Spark 3.1 or earlier, the `namespace` field was named `database` for the builtin catalog, and no change for the v2 catalogs. To restore the old schema with the builtin catalog, you can set `spark.sql.legacy.keepCommandOutputSchema` to `true`.

- In Spark 3.2, the output schema of `SHOW TBLPROPERTIES` becomes `key: string, value: string` whether you specify the table property key or not. In Spark 3.1 and earlier, the output schema of `SHOW TBLPROPERTIES` is `value: string` when you specify the table property key. To restore the old schema with the builtin catalog, you can set `spark.sql.legacy.keepCommandOutputSchema` to `true`.

- In Spark 3.2, the output schema of `DESCRIBE NAMESPACE` becomes `info_name: string, info_value: string`. In Spark 3.1 or earlier, the `info_name` field was named `database_description_item` and the `info_value` field was named `database_description_value` for the builtin catalog. To restore the old schema with the builtin catalog, you can set `spark.sql.legacy.keepCommandOutputSchema` to `true`.

- In Spark 3.2, table refreshing clears cached data of the table as well as of all its dependents such as views while keeping the dependents cached. The following commands perform table refreshing:
* `ALTER TABLE .. ADD PARTITION`
* `ALTER TABLE .. RENAME PARTITION`
Expand All @@ -62,8 +66,6 @@ license: |
In Spark 3.1 and earlier, table refreshing leaves dependents uncached.

- In Spark 3.2, the usage of `count(tblName.*)` is blocked to avoid producing ambiguous results. Because `count(*)` and `count(tblName.*)` will output differently if there is any null values. To restore the behavior before Spark 3.2, you can set `spark.sql.legacy.allowStarWithSingleTableIdentifierInCount` to `true`.

- In Spark 3.2, the output schema of `SHOW TBLPROPERTIES` becomes `key: string, value: string` whether you specify the table property key or not. In Spark 3.1 and earlier, the output schema of `SHOW TBLPROPERTIES` is `value: string` when you specify the table property key. To restore the old schema with the builtin catalog, you can set `spark.sql.legacy.keepCommandOutputSchema` to `true`.

- In Spark 3.2, we support typed literals in the partition spec of INSERT and ADD/DROP/RENAME PARTITION. For example, `ADD PARTITION(dt = date'2020-01-01')` adds a partition with date value `2020-01-01`. In Spark 3.1 and earlier, the partition value will be parsed as string value `date '2020-01-01'`, which is an illegal date value, and we add a partition with null value at the end.

Expand Down
46 changes: 0 additions & 46 deletions python/pyspark/pandas/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -54,56 +54,10 @@ def assert_python_version():
)


def assert_pyspark_version():
import logging

try:
import pyspark
except ImportError:
raise ImportError(
"Unable to import pyspark - consider doing a pip install with [spark] "
"extra to install pyspark with pip"
)
else:
pyspark_ver = getattr(pyspark, "__version__")
if pyspark_ver is None or LooseVersion(pyspark_ver) < LooseVersion("2.4"):
logging.warning(
'Found pyspark version "{}" installed. pyspark>=2.4.0 is recommended.'.format(
pyspark_ver if pyspark_ver is not None else "<unknown version>"
)
)


assert_python_version()
assert_pyspark_version()

import pyspark
import pyarrow

if LooseVersion(pyspark.__version__) < LooseVersion("3.0"):
if (
LooseVersion(pyarrow.__version__) >= LooseVersion("0.15")
and "ARROW_PRE_0_15_IPC_FORMAT" not in os.environ
):
import logging

logging.warning(
"'ARROW_PRE_0_15_IPC_FORMAT' environment variable was not set. It is required to "
"set this environment variable to '1' in both driver and executor sides if you use "
"pyarrow>=0.15 and pyspark<3.0. "
"Koalas will set it for you but it does not work if there is a Spark context already "
"launched."
)
# This is required to support PyArrow 0.15 in PySpark versions lower than 3.0.
# See SPARK-29367.
os.environ["ARROW_PRE_0_15_IPC_FORMAT"] = "1"
elif "ARROW_PRE_0_15_IPC_FORMAT" in os.environ:
raise RuntimeError(
"Please explicitly unset 'ARROW_PRE_0_15_IPC_FORMAT' environment variable in both "
"driver and executor sides. It is required to set this environment variable only "
"when you use pyarrow>=0.15 and pyspark<3.0."
)

if (
LooseVersion(pyarrow.__version__) >= LooseVersion("2.0.0")
and "PYARROW_IGNORE_TIMEZONE" not in os.environ
Expand Down
Loading