diff --git a/.github/workflows/clickhouse_be_trigger.yml b/.github/workflows/clickhouse_be_trigger.yml
index f19a328adcb3..19f9b55a0303 100644
--- a/.github/workflows/clickhouse_be_trigger.yml
+++ b/.github/workflows/clickhouse_be_trigger.yml
@@ -34,17 +34,21 @@ on:
jobs:
add-comment:
runs-on: ubuntu-latest
+ permissions: write-all
steps:
- name: Checkout code
uses: actions/checkout@v4
- name: Sleep for Dev PR workflow done
run: |
sleep 15
-
- name: Add comment to PR
- env:
- GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }}
- run: |
- COMMENT="Run Gluten Clickhouse CI"
- URL=$(jq -r .pull_request.comments_url "$GITHUB_EVENT_PATH")
- curl -H "Authorization: token ${GITHUB_TOKEN}" -X POST -d "{\"body\":\"$COMMENT\"}" "${URL}"
+ uses: actions/github-script@v7
+ with:
+ github-token: ${{ secrets.GITHUB_TOKEN }}
+ script: |
+ await github.rest.issues.createComment({
+ owner: context.repo.owner,
+ repo: context.repo.repo,
+ issue_number: context.payload.number,
+ body: "Run Gluten Clickhouse CI"
+ });
diff --git a/.github/workflows/velox_docker.yml b/.github/workflows/velox_docker.yml
index c310d5298f12..644fe62b932e 100644
--- a/.github/workflows/velox_docker.yml
+++ b/.github/workflows/velox_docker.yml
@@ -68,17 +68,16 @@ jobs:
if: ${{ steps.cache.outputs.cache-hit != 'true' }}
run: |
df -a
+ cd $GITHUB_WORKSPACE/
bash dev/ci-velox-buildstatic-centos-7.sh
- - name: Upload Artifact Native
- uses: actions/upload-artifact@v2
+ - uses: actions/upload-artifact@v2
with:
- path: ./cpp/build/releases/
name: velox-native-lib-centos-7-${{github.sha}}
- - name: Upload Artifact Arrow Jar
- uses: actions/upload-artifact@v2
+ path: ./cpp/build/releases/
+ - uses: actions/upload-artifact@v2
with:
+ name: arrow-jars-centos-7-${{github.sha}}
path: /root/.m2/repository/org/apache/arrow/
- name: velox-arrow-jar-centos-7-${{github.sha}}
run-tpc-test-ubuntu:
needs: build-native-lib-centos-7
@@ -118,7 +117,7 @@ jobs:
- name: Download All Arrow Jar Artifacts
uses: actions/download-artifact@v2
with:
- name: velox-arrow-jar-centos-7-${{github.sha}}
+ name: arrow-jars-centos-7-${{github.sha}}
path: /root/.m2/repository/org/apache/arrow/
- name: Setup tzdata
run: |
@@ -189,7 +188,7 @@ jobs:
- name: Download All Arrow Jar Artifacts
uses: actions/download-artifact@v2
with:
- name: velox-arrow-jar-centos-7-${{github.sha}}
+ name: arrow-jars-centos-7-${{github.sha}}
path: /root/.m2/repository/org/apache/arrow/
- name: Update mirror list
run: |
@@ -274,7 +273,7 @@ jobs:
- name: Download All Arrow Jar Artifacts
uses: actions/download-artifact@v2
with:
- name: velox-arrow-jar-centos-7-${{github.sha}}
+ name: arrow-jars-centos-7-${{github.sha}}
path: /home/runner/.m2/repository/org/apache/arrow/
- name: Setup java and maven
run: |
@@ -386,7 +385,7 @@ jobs:
- name: Download All Arrow Jar Artifacts
uses: actions/download-artifact@v2
with:
- name: velox-arrow-jar-centos-7-${{github.sha}}
+ name: arrow-jars-centos-7-${{github.sha}}
path: /home/runner/.m2/repository/org/apache/arrow/
- name: Setup java and maven
run: |
@@ -479,7 +478,7 @@ jobs:
- name: Download All Arrow Jar Artifacts
uses: actions/download-artifact@v2
with:
- name: velox-arrow-jar-centos-7-${{github.sha}}
+ name: arrow-jars-centos-7-${{github.sha}}
path: /root/.m2/repository/org/apache/arrow/
- name: Update mirror list
run: |
@@ -546,7 +545,7 @@ jobs:
- name: Download All Arrow Jar Artifacts
uses: actions/download-artifact@v2
with:
- name: velox-arrow-jar-centos-7-${{github.sha}}
+ name: arrow-jars-centos-7-${{github.sha}}
path: /root/.m2/repository/org/apache/arrow/
- name: Setup tzdata
run: |
@@ -583,7 +582,7 @@ jobs:
GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \
--local --preset=velox-with-celeborn --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=8 --iterations=1
- build-native-lib-centos-8:
+ run-CPP-test-UDF-test:
runs-on: ubuntu-20.04
container: ghcr.io/facebookincubator/velox-dev:centos8
steps:
@@ -606,50 +605,37 @@ jobs:
run: |
df -a
bash dev/ci-velox-buildshared-centos-8.sh
- - uses: actions/upload-artifact@v2
- with:
- name: velox-native-lib-centos-8-${{github.sha}}
- path: ./cpp/build/releases/
- - uses: actions/upload-artifact@v2
- with:
- name: udf-example-lib-centos-8-${{github.sha}}
- path: ./cpp/build/velox/udf/examples/
- - uses: actions/upload-artifact@v2
- with:
- name: benchmark-centos-8-${{github.sha}}
- path: ./cpp/build/velox/benchmarks/
- - uses: actions/upload-artifact@v2
- with:
- name: arrow-jars-centos-8-${{github.sha}}
- path: /root/.m2/repository/org/apache/arrow/
+ - name: Run CPP unit test
+ run: |
+ cd ./cpp/build && ctest -V
+ - name: Run CPP benchmark test
+ run: |
+ $MVN_CMD test -Pspark-3.5 -Pbackends-velox -pl backends-velox -am \
+ -DtagsToInclude="org.apache.gluten.tags.GenerateExample" -Dtest=none -DfailIfNoTests=false -Dexec.skip
+ # This test depends on example.json generated by the above mvn test.
+ cd cpp/build/velox/benchmarks && sudo chmod +x ./generic_benchmark
+ ./generic_benchmark --run-example --with-shuffle --threads 1 --iterations 1
+ - name: Run UDF test
+ run: |
+ # Depends on --build_example=ON.
+ $MVN_CMD test -Pspark-3.5 -Pbackends-velox -Piceberg -Pdelta -DtagsToExclude=None \
+ -DtagsToInclude=org.apache.gluten.tags.UDFTest
run-spark-test-spark32:
- needs: build-native-lib-centos-8
+ needs: build-native-lib-centos-7
runs-on: ubuntu-20.04
container: ghcr.io/facebookincubator/velox-dev:centos8
env:
CCACHE_DIR: "${{ github.workspace }}/.ccache"
steps:
- uses: actions/checkout@v2
- - name: Download All Artifacts
- uses: actions/download-artifact@v2
+ - uses: actions/download-artifact@v2
with:
- name: velox-native-lib-centos-8-${{github.sha}}
+ name: velox-native-lib-centos-7-${{github.sha}}
path: ./cpp/build/releases
- - name: Download UDF Example Lib
- uses: actions/download-artifact@v2
+ - uses: actions/download-artifact@v2
with:
- name: udf-example-lib-centos-8-${{github.sha}}
- path: ./cpp/build/velox/udf/examples/
- - name: Download Benchmark
- uses: actions/download-artifact@v2
- with:
- name: benchmark-centos-8-${{github.sha}}
- path: ./cpp/build/velox/benchmarks/
- - name: Download Arrow Jars
- uses: actions/download-artifact@v2
- with:
- name: arrow-jars-centos-8-${{github.sha}}
+ name: arrow-jars-centos-7-${{github.sha}}
path: /root/.m2/repository/org/apache/arrow/
- name: Update mirror list
run: |
@@ -693,23 +679,18 @@ jobs:
run: |
cd $GITHUB_WORKSPACE/
export SPARK_SCALA_VERSION=2.12
- $MVN_CMD clean install -Pspark-3.2 -Pspark-ut -Pbackends-velox -Pceleborn -Piceberg -Pdelta -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark32/spark_home/" -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags && \
- $MVN_CMD test -Pspark-3.2 -Pbackends-velox -Piceberg -Pdelta -DtagsToExclude=None -DtagsToInclude=org.apache.gluten.tags.UDFTest
+ $MVN_CMD clean install -Pspark-3.2 -Pspark-ut -Pbackends-velox -Pceleborn -Piceberg \
+ -Pdelta -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark32/spark_home/" \
+ -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags
- name: Upload golden files
if: failure()
uses: actions/upload-artifact@v4
with:
name: golden-files-spark32
path: /tmp/tpch-approved-plan/**
- - name: Gluten CPP Benchmark Test
- run: |
- # This test depends on example.json generated by the above mvn test.
- cd $GITHUB_WORKSPACE/cpp/build/velox/benchmarks && \
- sudo chmod +x ./generic_benchmark && \
- ./generic_benchmark --run-example --with-shuffle --threads 1 --iterations 1
run-spark-test-spark32-slow:
- needs: build-native-lib-centos-8
+ needs: build-native-lib-centos-7
runs-on: ubuntu-20.04
container: ghcr.io/facebookincubator/velox-dev:centos8
env:
@@ -719,12 +700,12 @@ jobs:
- name: Download All Artifacts
uses: actions/download-artifact@v2
with:
- name: velox-native-lib-centos-8-${{github.sha}}
+ name: velox-native-lib-centos-7-${{github.sha}}
path: ./cpp/build/releases
- name: Download Arrow Jars
uses: actions/download-artifact@v2
with:
- name: arrow-jars-centos-8-${{github.sha}}
+ name: arrow-jars-centos-7-${{github.sha}}
path: /root/.m2/repository/org/apache/arrow/
- name: Update mirror list
run: |
@@ -759,7 +740,7 @@ jobs:
$MVN_CMD clean install -Pspark-3.2 -Pspark-ut -Pbackends-velox -Pceleborn -Piceberg -Pdelta -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark32/spark_home/" -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest
run-spark-test-spark33:
- needs: build-native-lib-centos-8
+ needs: build-native-lib-centos-7
runs-on: ubuntu-20.04
container: ghcr.io/facebookincubator/velox-dev:centos8
env:
@@ -769,17 +750,12 @@ jobs:
- name: Download All Artifacts
uses: actions/download-artifact@v2
with:
- name: velox-native-lib-centos-8-${{github.sha}}
+ name: velox-native-lib-centos-7-${{github.sha}}
path: ./cpp/build/releases
- - name: Download UDF Example Lib
- uses: actions/download-artifact@v2
- with:
- name: udf-example-lib-centos-8-${{github.sha}}
- path: ./cpp/build/velox/udf/examples/
- name: Download Arrow Jars
uses: actions/download-artifact@v2
with:
- name: arrow-jars-centos-8-${{github.sha}}
+ name: arrow-jars-centos-7-${{github.sha}}
path: /root/.m2/repository/org/apache/arrow/
- name: Update mirror list
run: |
@@ -823,8 +799,9 @@ jobs:
run: |
cd $GITHUB_WORKSPACE/
export SPARK_SCALA_VERSION=2.12
- $MVN_CMD clean install -Pspark-3.3 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark33/spark_home/" -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags && \
- $MVN_CMD test -Pspark-3.3 -Pbackends-velox -Piceberg -Pdelta -DtagsToExclude=None -DtagsToInclude=org.apache.gluten.tags.UDFTest
+ $MVN_CMD clean install -Pspark-3.3 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \
+ -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark33/spark_home/" \
+ -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags
- name: Upload golden files
if: failure()
uses: actions/upload-artifact@v4
@@ -834,7 +811,7 @@ jobs:
run-spark-test-spark33-slow:
- needs: build-native-lib-centos-8
+ needs: build-native-lib-centos-7
runs-on: ubuntu-20.04
container: ghcr.io/facebookincubator/velox-dev:centos8
env:
@@ -844,12 +821,12 @@ jobs:
- name: Download All Artifacts
uses: actions/download-artifact@v2
with:
- name: velox-native-lib-centos-8-${{github.sha}}
+ name: velox-native-lib-centos-7-${{github.sha}}
path: ./cpp/build/releases
- name: Download Arrow Jars
uses: actions/download-artifact@v2
with:
- name: arrow-jars-centos-8-${{github.sha}}
+ name: arrow-jars-centos-7-${{github.sha}}
path: /root/.m2/repository/org/apache/arrow/
- name: Update mirror list
run: |
@@ -881,10 +858,12 @@ jobs:
- name: Build and Run unit test for Spark 3.3.1 (slow tests)
run: |
cd $GITHUB_WORKSPACE/
- $MVN_CMD clean install -Pspark-3.3 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark33/spark_home/" -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest
+ $MVN_CMD clean install -Pspark-3.3 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \
+ -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark33/spark_home/" \
+ -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest
run-spark-test-spark34:
- needs: build-native-lib-centos-8
+ needs: build-native-lib-centos-7
runs-on: ubuntu-20.04
container: ghcr.io/facebookincubator/velox-dev:centos8
env:
@@ -894,17 +873,12 @@ jobs:
- name: Download All Artifacts
uses: actions/download-artifact@v2
with:
- name: velox-native-lib-centos-8-${{github.sha}}
+ name: velox-native-lib-centos-7-${{github.sha}}
path: ./cpp/build/releases
- - name: Download UDF Example Lib
- uses: actions/download-artifact@v2
- with:
- name: udf-example-lib-centos-8-${{github.sha}}
- path: ./cpp/build/velox/udf/examples/
- name: Download Arrow Jars
uses: actions/download-artifact@v2
with:
- name: arrow-jars-centos-8-${{github.sha}}
+ name: arrow-jars-centos-7-${{github.sha}}
path: /root/.m2/repository/org/apache/arrow/
- name: Update mirror list
run: |
@@ -948,8 +922,9 @@ jobs:
run: |
cd $GITHUB_WORKSPACE/
export SPARK_SCALA_VERSION=2.12
- $MVN_CMD clean install -Pspark-3.4 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark34/spark_home/" -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags && \
- $MVN_CMD test -Pspark-3.4 -Pbackends-velox -Piceberg -Pdelta -DtagsToExclude=None -DtagsToInclude=org.apache.gluten.tags.UDFTest
+ $MVN_CMD clean install -Pspark-3.4 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \
+ -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark34/spark_home/" \
+ -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags
- name: Upload golden files
if: failure()
uses: actions/upload-artifact@v4
@@ -959,7 +934,7 @@ jobs:
run-spark-test-spark34-slow:
- needs: build-native-lib-centos-8
+ needs: build-native-lib-centos-7
runs-on: ubuntu-20.04
container: ghcr.io/facebookincubator/velox-dev:centos8
env:
@@ -969,12 +944,12 @@ jobs:
- name: Download All Artifacts
uses: actions/download-artifact@v2
with:
- name: velox-native-lib-centos-8-${{github.sha}}
+ name: velox-native-lib-centos-7-${{github.sha}}
path: ./cpp/build/releases
- name: Download Arrow Jars
uses: actions/download-artifact@v2
with:
- name: arrow-jars-centos-8-${{github.sha}}
+ name: arrow-jars-centos-7-${{github.sha}}
path: /root/.m2/repository/org/apache/arrow/
- name: Update mirror list
run: |
@@ -1006,10 +981,12 @@ jobs:
- name: Build and Run unit test for Spark 3.4.2 (slow tests)
run: |
cd $GITHUB_WORKSPACE/
- $MVN_CMD clean install -Pspark-3.4 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark34/spark_home/" -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest
+ $MVN_CMD clean install -Pspark-3.4 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \
+ -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark34/spark_home/" \
+ -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest
run-spark-test-spark35:
- needs: build-native-lib-centos-8
+ needs: build-native-lib-centos-7
runs-on: ubuntu-20.04
container: ghcr.io/facebookincubator/velox-dev:centos8
env:
@@ -1019,17 +996,12 @@ jobs:
- name: Download All Artifacts
uses: actions/download-artifact@v2
with:
- name: velox-native-lib-centos-8-${{github.sha}}
+ name: velox-native-lib-centos-7-${{github.sha}}
path: ./cpp/build/releases
- - name: Download UDF Example Lib
- uses: actions/download-artifact@v2
- with:
- name: udf-example-lib-centos-8-${{github.sha}}
- path: ./cpp/build/velox/udf/examples/
- name: Download Arrow Jars
uses: actions/download-artifact@v2
with:
- name: arrow-jars-centos-8-${{github.sha}}
+ name: arrow-jars-centos-7-${{github.sha}}
path: /root/.m2/repository/org/apache/arrow/
- name: Update mirror list
run: |
@@ -1073,8 +1045,9 @@ jobs:
run: |
cd $GITHUB_WORKSPACE/
export SPARK_SCALA_VERSION=2.12
- $MVN_CMD clean install -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/" -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags && \
- $MVN_CMD test -Pspark-3.5 -Pbackends-velox -Piceberg -Pdelta -DtagsToExclude=None -DtagsToInclude=org.apache.gluten.tags.UDFTest
+ $MVN_CMD clean install -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \
+ -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/" \
+ -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags
- name: Upload golden files
if: failure()
uses: actions/upload-artifact@v4
@@ -1083,7 +1056,7 @@ jobs:
path: /tmp/tpch-approved-plan/**
run-spark-test-spark35-scala213:
- needs: build-native-lib-centos-8
+ needs: build-native-lib-centos-7
runs-on: ubuntu-20.04
container: ghcr.io/facebookincubator/velox-dev:centos8
env:
@@ -1093,17 +1066,12 @@ jobs:
- name: Download All Artifacts
uses: actions/download-artifact@v2
with:
- name: velox-native-lib-centos-8-${{github.sha}}
+ name: velox-native-lib-centos-7-${{github.sha}}
path: ./cpp/build/releases
- - name: Download UDF Example Lib
- uses: actions/download-artifact@v2
- with:
- name: udf-example-lib-centos-8-${{github.sha}}
- path: ./cpp/build/velox/udf/examples/
- name: Download Arrow Jars
uses: actions/download-artifact@v2
with:
- name: arrow-jars-centos-8-${{github.sha}}
+ name: arrow-jars-centos-7-${{github.sha}}
path: /root/.m2/repository/org/apache/arrow/
- name: Update mirror list
run: |
@@ -1147,11 +1115,12 @@ jobs:
run: |
cd $GITHUB_WORKSPACE/
export SPARK_SCALA_VERSION=2.13
- $MVN_CMD clean install -Pspark-3.5 -Pscala-2.13 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/" -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags && \
- $MVN_CMD test -Pspark-3.5 -Pscala-2.13 -Pbackends-velox -Piceberg -Pdelta -DtagsToExclude=None -DtagsToInclude=org.apache.gluten.tags.UDFTest
+ $MVN_CMD clean install -Pspark-3.5 -Pscala-2.13 -Pbackends-velox -Pceleborn -Piceberg \
+ -Pdelta -Pspark-ut -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/" \
+ -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags
run-spark-test-spark35-slow:
- needs: build-native-lib-centos-8
+ needs: build-native-lib-centos-7
runs-on: ubuntu-20.04
container: ghcr.io/facebookincubator/velox-dev:centos8
env:
@@ -1161,12 +1130,12 @@ jobs:
- name: Download All Artifacts
uses: actions/download-artifact@v2
with:
- name: velox-native-lib-centos-8-${{github.sha}}
+ name: velox-native-lib-centos-7-${{github.sha}}
path: ./cpp/build/releases
- name: Download Arrow Jars
uses: actions/download-artifact@v2
with:
- name: arrow-jars-centos-8-${{github.sha}}
+ name: arrow-jars-centos-7-${{github.sha}}
path: /root/.m2/repository/org/apache/arrow/
- name: Update mirror list
run: |
@@ -1204,4 +1173,6 @@ jobs:
- name: Build and Run unit test for Spark 3.5.1 (slow tests)
run: |
cd $GITHUB_WORKSPACE/
- $MVN_CMD clean install -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/" -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest
+ $MVN_CMD clean install -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \
+ -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/" \
+ -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala
index 9884a0c6ef39..41ffbdb58354 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala
@@ -53,6 +53,7 @@ class CHBackend extends Backend {
override def validatorApi(): ValidatorApi = new CHValidatorApi
override def metricsApi(): MetricsApi = new CHMetricsApi
override def listenerApi(): ListenerApi = new CHListenerApi
+ override def ruleApi(): RuleApi = new CHRuleApi
override def settings(): BackendSettingsApi = CHBackendSettings
}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala
new file mode 100644
index 000000000000..177d6a6f0f4c
--- /dev/null
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala
@@ -0,0 +1,111 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.backendsapi.clickhouse
+
+import org.apache.gluten.backendsapi.RuleApi
+import org.apache.gluten.extension._
+import org.apache.gluten.extension.columnar._
+import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast, TransformPreOverrides}
+import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager
+import org.apache.gluten.extension.columnar.transition.{InsertTransitions, RemoveTransitions}
+import org.apache.gluten.extension.injector.{RuleInjector, SparkInjector}
+import org.apache.gluten.extension.injector.GlutenInjector.{LegacyInjector, RasInjector}
+import org.apache.gluten.parser.GlutenClickhouseSqlParser
+import org.apache.gluten.sql.shims.SparkShimLoader
+
+import org.apache.spark.sql.catalyst.{CHAggregateFunctionRewriteRule, EqualToRewrite}
+import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, GlutenFallbackReporter}
+import org.apache.spark.util.SparkPlanRules
+
+class CHRuleApi extends RuleApi {
+ import CHRuleApi._
+ override def injectRules(injector: RuleInjector): Unit = {
+ injectSpark(injector.spark)
+ injectLegacy(injector.gluten.legacy)
+ injectRas(injector.gluten.ras)
+ }
+}
+
+private object CHRuleApi {
+ def injectSpark(injector: SparkInjector): Unit = {
+ // Regular Spark rules.
+ injector.injectQueryStagePrepRule(FallbackBroadcastHashJoinPrepQueryStage.apply)
+ injector.injectParser(
+ (spark, parserInterface) => new GlutenClickhouseSqlParser(spark, parserInterface))
+ injector.injectResolutionRule(
+ spark => new RewriteToDateExpresstionRule(spark, spark.sessionState.conf))
+ injector.injectResolutionRule(
+ spark => new RewriteDateTimestampComparisonRule(spark, spark.sessionState.conf))
+ injector.injectOptimizerRule(
+ spark => new CommonSubexpressionEliminateRule(spark, spark.sessionState.conf))
+ injector.injectOptimizerRule(spark => CHAggregateFunctionRewriteRule(spark))
+ injector.injectOptimizerRule(_ => CountDistinctWithoutExpand)
+ injector.injectOptimizerRule(_ => EqualToRewrite)
+ }
+
+ def injectLegacy(injector: LegacyInjector): Unit = {
+ // Gluten columnar: Transform rules.
+ injector.injectTransform(_ => RemoveTransitions)
+ injector.injectTransform(c => FallbackOnANSIMode.apply(c.session))
+ injector.injectTransform(c => FallbackMultiCodegens.apply(c.session))
+ injector.injectTransform(c => PlanOneRowRelation.apply(c.session))
+ injector.injectTransform(_ => RewriteSubqueryBroadcast())
+ injector.injectTransform(c => FallbackBroadcastHashJoin.apply(c.session))
+ injector.injectTransform(_ => FallbackEmptySchemaRelation())
+ injector.injectTransform(c => MergeTwoPhasesHashBaseAggregate.apply(c.session))
+ injector.injectTransform(_ => RewriteSparkPlanRulesManager())
+ injector.injectTransform(_ => AddFallbackTagRule())
+ injector.injectTransform(_ => TransformPreOverrides())
+ injector.injectTransform(_ => RemoveNativeWriteFilesSortAndProject())
+ injector.injectTransform(c => RewriteTransformer.apply(c.session))
+ injector.injectTransform(_ => EnsureLocalSortRequirements)
+ injector.injectTransform(_ => EliminateLocalSort)
+ injector.injectTransform(_ => CollapseProjectExecTransformer)
+ injector.injectTransform(c => RewriteSortMergeJoinToHashJoinRule.apply(c.session))
+ injector.injectTransform(
+ c => SparkPlanRules.extendedColumnarRule(c.conf.extendedColumnarTransformRules)(c.session))
+ injector.injectTransform(c => InsertTransitions(c.outputsColumnar))
+
+ // Gluten columnar: Fallback policies.
+ injector.injectFallbackPolicy(
+ c => ExpandFallbackPolicy(c.ac.isAdaptiveContext(), c.ac.originalPlan()))
+
+ // Gluten columnar: Post rules.
+ injector.injectPost(c => RemoveTopmostColumnarToRow(c.session, c.ac.isAdaptiveContext()))
+ SparkShimLoader.getSparkShims
+ .getExtendedColumnarPostRules()
+ .foreach(each => injector.injectPost(c => each(c.session)))
+ injector.injectPost(c => ColumnarCollapseTransformStages(c.conf))
+ injector.injectTransform(
+ c => SparkPlanRules.extendedColumnarRule(c.conf.extendedColumnarPostRules)(c.session))
+
+ // Gluten columnar: Final rules.
+ injector.injectFinal(c => RemoveGlutenTableCacheColumnarToRow(c.session))
+ injector.injectFinal(c => GlutenFallbackReporter(c.conf, c.session))
+ injector.injectFinal(_ => RemoveFallbackTagRule())
+ }
+
+ def injectRas(injector: RasInjector): Unit = {
+ // CH backend doesn't work with RAS at the moment. Inject a rule that aborts any
+ // execution calls.
+ injector.inject(
+ _ =>
+ new SparkPlanRules.AbortRule(
+ "Clickhouse backend doesn't yet have RAS support, please try disabling RAS and" +
+ " rerunning the application"))
+ }
+}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
index 8fdc2645a5fb..02b4777e7120 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
@@ -21,11 +21,9 @@ import org.apache.gluten.backendsapi.{BackendsApiManager, SparkPlanExecApi}
import org.apache.gluten.exception.GlutenNotSupportException
import org.apache.gluten.execution._
import org.apache.gluten.expression._
-import org.apache.gluten.extension.{CommonSubexpressionEliminateRule, CountDistinctWithoutExpand, FallbackBroadcastHashJoin, FallbackBroadcastHashJoinPrepQueryStage, RewriteDateTimestampComparisonRule, RewriteSortMergeJoinToHashJoinRule, RewriteToDateExpresstionRule}
import org.apache.gluten.extension.columnar.AddFallbackTagRule
import org.apache.gluten.extension.columnar.MiscColumnarRules.TransformPreOverrides
import org.apache.gluten.extension.columnar.transition.Convention
-import org.apache.gluten.parser.GlutenClickhouseSqlParser
import org.apache.gluten.sql.shims.SparkShimLoader
import org.apache.gluten.substrait.expression.{ExpressionBuilder, ExpressionNode, WindowFunctionNode}
import org.apache.gluten.utils.{CHJoinValidateUtil, UnknownJoinStrategy}
@@ -36,18 +34,13 @@ import org.apache.spark.rdd.RDD
import org.apache.spark.serializer.Serializer
import org.apache.spark.shuffle.{GenShuffleWriterParameters, GlutenShuffleWriterWrapper, HashPartitioningWrapper}
import org.apache.spark.shuffle.utils.CHShuffleUtil
-import org.apache.spark.sql.{SparkSession, Strategy}
-import org.apache.spark.sql.catalyst.{CHAggregateFunctionRewriteRule, EqualToRewrite}
import org.apache.spark.sql.catalyst.catalog.BucketSpec
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, CollectList, CollectSet}
import org.apache.spark.sql.catalyst.optimizer.BuildSide
-import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.plans.JoinType
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, HashPartitioning, Partitioning, RangePartitioning}
-import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.delta.files.TahoeFileIndex
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.adaptive.AQEShuffleReadExec
@@ -549,82 +542,6 @@ class CHSparkPlanExecApi extends SparkPlanExecApi {
ClickHouseBuildSideRelation(mode, newOutput, batches.flatten, rowCount, newBuildKeys)
}
- /**
- * Generate extended DataSourceV2 Strategies. Currently only for ClickHouse backend.
- *
- * @return
- */
- override def genExtendedDataSourceV2Strategies(): List[SparkSession => Strategy] = {
- List.empty
- }
-
- /**
- * Generate extended query stage preparation rules.
- *
- * @return
- */
- override def genExtendedQueryStagePrepRules(): List[SparkSession => Rule[SparkPlan]] = {
- List(spark => FallbackBroadcastHashJoinPrepQueryStage(spark))
- }
-
- /**
- * Generate extended Analyzers. Currently only for ClickHouse backend.
- *
- * @return
- */
- override def genExtendedAnalyzers(): List[SparkSession => Rule[LogicalPlan]] = {
- List(
- spark => new RewriteToDateExpresstionRule(spark, spark.sessionState.conf),
- spark => new RewriteDateTimestampComparisonRule(spark, spark.sessionState.conf))
- }
-
- /**
- * Generate extended Optimizers.
- *
- * @return
- */
- override def genExtendedOptimizers(): List[SparkSession => Rule[LogicalPlan]] = {
- List(
- spark => new CommonSubexpressionEliminateRule(spark, spark.sessionState.conf),
- spark => CHAggregateFunctionRewriteRule(spark),
- _ => CountDistinctWithoutExpand,
- _ => EqualToRewrite
- )
- }
-
- /**
- * Generate extended columnar pre-rules, in the validation phase.
- *
- * @return
- */
- override def genExtendedColumnarValidationRules(): List[SparkSession => Rule[SparkPlan]] =
- List(spark => FallbackBroadcastHashJoin(spark))
-
- /**
- * Generate extended columnar pre-rules.
- *
- * @return
- */
- override def genExtendedColumnarTransformRules(): List[SparkSession => Rule[SparkPlan]] =
- List(spark => RewriteSortMergeJoinToHashJoinRule(spark))
-
- override def genInjectPostHocResolutionRules(): List[SparkSession => Rule[LogicalPlan]] = {
- List()
- }
-
- /**
- * Generate extended Strategies.
- *
- * @return
- */
- override def genExtendedStrategies(): List[SparkSession => Strategy] =
- List()
-
- override def genInjectExtendedParser()
- : List[(SparkSession, ParserInterface) => ParserInterface] = {
- List((spark, parserInterface) => new GlutenClickhouseSqlParser(spark, parserInterface))
- }
-
/** Define backend specfic expression mappings. */
override def extraExpressionMappings: Seq[Sig] = {
List(
diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala
index d32911f4a4c7..21175f20eb64 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala
@@ -55,6 +55,7 @@ class VeloxBackend extends Backend {
override def validatorApi(): ValidatorApi = new VeloxValidatorApi
override def metricsApi(): MetricsApi = new VeloxMetricsApi
override def listenerApi(): ListenerApi = new VeloxListenerApi
+ override def ruleApi(): RuleApi = new VeloxRuleApi
override def settings(): BackendSettingsApi = VeloxBackendSettings
}
diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala
new file mode 100644
index 000000000000..645407be8be5
--- /dev/null
+++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala
@@ -0,0 +1,133 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.backendsapi.velox
+
+import org.apache.gluten.backendsapi.RuleApi
+import org.apache.gluten.datasource.ArrowConvertorRule
+import org.apache.gluten.extension._
+import org.apache.gluten.extension.columnar._
+import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast, TransformPreOverrides}
+import org.apache.gluten.extension.columnar.enumerated.EnumeratedTransform
+import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager
+import org.apache.gluten.extension.columnar.transition.{InsertTransitions, RemoveTransitions}
+import org.apache.gluten.extension.injector.{RuleInjector, SparkInjector}
+import org.apache.gluten.extension.injector.GlutenInjector.{LegacyInjector, RasInjector}
+import org.apache.gluten.sql.shims.SparkShimLoader
+
+import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, GlutenFallbackReporter}
+import org.apache.spark.sql.expression.UDFResolver
+import org.apache.spark.util.SparkPlanRules
+
+class VeloxRuleApi extends RuleApi {
+ import VeloxRuleApi._
+
+ override def injectRules(injector: RuleInjector): Unit = {
+ injectSpark(injector.spark)
+ injectLegacy(injector.gluten.legacy)
+ injectRas(injector.gluten.ras)
+ }
+}
+
+private object VeloxRuleApi {
+ def injectSpark(injector: SparkInjector): Unit = {
+ // Regular Spark rules.
+ injector.injectOptimizerRule(CollectRewriteRule.apply)
+ injector.injectOptimizerRule(HLLRewriteRule.apply)
+ UDFResolver.getFunctionSignatures.foreach(injector.injectFunction)
+ injector.injectPostHocResolutionRule(ArrowConvertorRule.apply)
+ }
+
+ def injectLegacy(injector: LegacyInjector): Unit = {
+ // Gluten columnar: Transform rules.
+ injector.injectTransform(_ => RemoveTransitions)
+ injector.injectTransform(c => FallbackOnANSIMode.apply(c.session))
+ injector.injectTransform(c => FallbackMultiCodegens.apply(c.session))
+ injector.injectTransform(c => PlanOneRowRelation.apply(c.session))
+ injector.injectTransform(_ => RewriteSubqueryBroadcast())
+ injector.injectTransform(c => BloomFilterMightContainJointRewriteRule.apply(c.session))
+ injector.injectTransform(c => ArrowScanReplaceRule.apply(c.session))
+ injector.injectTransform(_ => FallbackEmptySchemaRelation())
+ injector.injectTransform(c => MergeTwoPhasesHashBaseAggregate.apply(c.session))
+ injector.injectTransform(_ => RewriteSparkPlanRulesManager())
+ injector.injectTransform(_ => AddFallbackTagRule())
+ injector.injectTransform(_ => TransformPreOverrides())
+ injector.injectTransform(_ => RemoveNativeWriteFilesSortAndProject())
+ injector.injectTransform(c => RewriteTransformer.apply(c.session))
+ injector.injectTransform(_ => EnsureLocalSortRequirements)
+ injector.injectTransform(_ => EliminateLocalSort)
+ injector.injectTransform(_ => CollapseProjectExecTransformer)
+ injector.injectTransform(c => FlushableHashAggregateRule.apply(c.session))
+ injector.injectTransform(
+ c => SparkPlanRules.extendedColumnarRule(c.conf.extendedColumnarTransformRules)(c.session))
+ injector.injectTransform(c => InsertTransitions(c.outputsColumnar))
+
+ // Gluten columnar: Fallback policies.
+ injector.injectFallbackPolicy(
+ c => ExpandFallbackPolicy(c.ac.isAdaptiveContext(), c.ac.originalPlan()))
+
+ // Gluten columnar: Post rules.
+ injector.injectPost(c => RemoveTopmostColumnarToRow(c.session, c.ac.isAdaptiveContext()))
+ SparkShimLoader.getSparkShims
+ .getExtendedColumnarPostRules()
+ .foreach(each => injector.injectPost(c => each(c.session)))
+ injector.injectPost(c => ColumnarCollapseTransformStages(c.conf))
+ injector.injectTransform(
+ c => SparkPlanRules.extendedColumnarRule(c.conf.extendedColumnarPostRules)(c.session))
+
+ // Gluten columnar: Final rules.
+ injector.injectFinal(c => RemoveGlutenTableCacheColumnarToRow(c.session))
+ injector.injectFinal(c => GlutenFallbackReporter(c.conf, c.session))
+ injector.injectFinal(_ => RemoveFallbackTagRule())
+ }
+
+ def injectRas(injector: RasInjector): Unit = {
+ // Gluten RAS: Pre rules.
+ injector.inject(_ => RemoveTransitions)
+ injector.inject(c => FallbackOnANSIMode.apply(c.session))
+ injector.inject(c => PlanOneRowRelation.apply(c.session))
+ injector.inject(_ => FallbackEmptySchemaRelation())
+ injector.inject(_ => RewriteSubqueryBroadcast())
+ injector.inject(c => BloomFilterMightContainJointRewriteRule.apply(c.session))
+ injector.inject(c => ArrowScanReplaceRule.apply(c.session))
+ injector.inject(c => MergeTwoPhasesHashBaseAggregate.apply(c.session))
+
+ // Gluten RAS: The RAS rule.
+ injector.inject(c => EnumeratedTransform(c.session, c.outputsColumnar))
+
+ // Gluten RAS: Post rules.
+ injector.inject(_ => RemoveTransitions)
+ injector.inject(_ => RemoveNativeWriteFilesSortAndProject())
+ injector.inject(c => RewriteTransformer.apply(c.session))
+ injector.inject(_ => EnsureLocalSortRequirements)
+ injector.inject(_ => EliminateLocalSort)
+ injector.inject(_ => CollapseProjectExecTransformer)
+ injector.inject(c => FlushableHashAggregateRule.apply(c.session))
+ injector.inject(
+ c => SparkPlanRules.extendedColumnarRule(c.conf.extendedColumnarTransformRules)(c.session))
+ injector.inject(c => InsertTransitions(c.outputsColumnar))
+ injector.inject(c => RemoveTopmostColumnarToRow(c.session, c.ac.isAdaptiveContext()))
+ SparkShimLoader.getSparkShims
+ .getExtendedColumnarPostRules()
+ .foreach(each => injector.inject(c => each(c.session)))
+ injector.inject(c => ColumnarCollapseTransformStages(c.conf))
+ injector.inject(
+ c => SparkPlanRules.extendedColumnarRule(c.conf.extendedColumnarPostRules)(c.session))
+ injector.inject(c => RemoveGlutenTableCacheColumnarToRow(c.session))
+ injector.inject(c => GlutenFallbackReporter(c.conf, c.session))
+ injector.inject(_ => RemoveFallbackTagRule())
+ }
+}
diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
index fd0fc62dcbb6..bd390004feda 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
@@ -18,12 +18,10 @@ package org.apache.gluten.backendsapi.velox
import org.apache.gluten.GlutenConfig
import org.apache.gluten.backendsapi.SparkPlanExecApi
-import org.apache.gluten.datasource.ArrowConvertorRule
import org.apache.gluten.exception.GlutenNotSupportException
import org.apache.gluten.execution._
import org.apache.gluten.expression._
import org.apache.gluten.expression.aggregate.{HLLAdapter, VeloxBloomFilterAggregate, VeloxCollectList, VeloxCollectSet}
-import org.apache.gluten.extension._
import org.apache.gluten.extension.columnar.FallbackTags
import org.apache.gluten.extension.columnar.transition.Convention
import org.apache.gluten.extension.columnar.transition.ConventionFunc.BatchOverride
@@ -36,18 +34,13 @@ import org.apache.spark.rdd.RDD
import org.apache.spark.serializer.Serializer
import org.apache.spark.shuffle.{GenShuffleWriterParameters, GlutenShuffleWriterWrapper}
import org.apache.spark.shuffle.utils.ShuffleUtil
-import org.apache.spark.sql.{SparkSession, Strategy}
-import org.apache.spark.sql.catalyst.FunctionIdentifier
-import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder
import org.apache.spark.sql.catalyst.catalog.BucketSpec
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.optimizer.BuildSide
import org.apache.spark.sql.catalyst.plans.JoinType
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.plans.physical._
-import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec
import org.apache.spark.sql.execution.datasources.FileFormat
@@ -56,7 +49,7 @@ import org.apache.spark.sql.execution.joins.{BuildSideRelation, HashedRelationBr
import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.execution.python.ArrowEvalPythonExec
import org.apache.spark.sql.execution.utils.ExecUtil
-import org.apache.spark.sql.expression.{UDFExpression, UDFResolver, UserDefinedAggregateFunction}
+import org.apache.spark.sql.expression.{UDFExpression, UserDefinedAggregateFunction}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.sql.vectorized.ColumnarBatch
@@ -65,8 +58,6 @@ import org.apache.commons.lang3.ClassUtils
import javax.ws.rs.core.UriBuilder
-import scala.collection.mutable.ListBuffer
-
class VeloxSparkPlanExecApi extends SparkPlanExecApi {
/** The columnar-batch type this backend is using. */
@@ -760,74 +751,6 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
}
}
- /**
- * * Rules and strategies.
- */
-
- /**
- * Generate extended DataSourceV2 Strategy.
- *
- * @return
- */
- override def genExtendedDataSourceV2Strategies(): List[SparkSession => Strategy] = List()
-
- /**
- * Generate extended query stage preparation rules.
- *
- * @return
- */
- override def genExtendedQueryStagePrepRules(): List[SparkSession => Rule[SparkPlan]] = List()
-
- /**
- * Generate extended Analyzer.
- *
- * @return
- */
- override def genExtendedAnalyzers(): List[SparkSession => Rule[LogicalPlan]] = List()
-
- /**
- * Generate extended Optimizer. Currently only for Velox backend.
- *
- * @return
- */
- override def genExtendedOptimizers(): List[SparkSession => Rule[LogicalPlan]] =
- List(CollectRewriteRule.apply, HLLRewriteRule.apply)
-
- /**
- * Generate extended columnar pre-rules, in the validation phase.
- *
- * @return
- */
- override def genExtendedColumnarValidationRules(): List[SparkSession => Rule[SparkPlan]] = {
- List(BloomFilterMightContainJointRewriteRule.apply, ArrowScanReplaceRule.apply)
- }
-
- /**
- * Generate extended columnar pre-rules.
- *
- * @return
- */
- override def genExtendedColumnarTransformRules(): List[SparkSession => Rule[SparkPlan]] = {
- val buf: ListBuffer[SparkSession => Rule[SparkPlan]] = ListBuffer()
- if (GlutenConfig.getConf.enableVeloxFlushablePartialAggregation) {
- buf += FlushableHashAggregateRule.apply
- }
- buf.result
- }
-
- override def genInjectPostHocResolutionRules(): List[SparkSession => Rule[LogicalPlan]] = {
- List(ArrowConvertorRule)
- }
-
- /**
- * Generate extended Strategy.
- *
- * @return
- */
- override def genExtendedStrategies(): List[SparkSession => Strategy] = {
- List()
- }
-
/** Define backend specfic expression mappings. */
override def extraExpressionMappings: Seq[Sig] = {
Seq(
@@ -844,11 +767,6 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
)
}
- override def genInjectedFunctions()
- : Seq[(FunctionIdentifier, ExpressionInfo, FunctionBuilder)] = {
- UDFResolver.getFunctionSignatures
- }
-
override def rewriteSpillPath(path: String): String = {
val fs = GlutenConfig.getConf.veloxSpillFileSystem
fs match {
diff --git a/backends-velox/src/main/scala/org/apache/gluten/extension/FlushableHashAggregateRule.scala b/backends-velox/src/main/scala/org/apache/gluten/extension/FlushableHashAggregateRule.scala
index 3137d6e6aef5..04bdbe1efb51 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/extension/FlushableHashAggregateRule.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/extension/FlushableHashAggregateRule.scala
@@ -16,6 +16,7 @@
*/
package org.apache.gluten.extension
+import org.apache.gluten.GlutenConfig
import org.apache.gluten.execution._
import org.apache.spark.sql.SparkSession
@@ -31,27 +32,32 @@ import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike
*/
case class FlushableHashAggregateRule(session: SparkSession) extends Rule[SparkPlan] {
import FlushableHashAggregateRule._
- override def apply(plan: SparkPlan): SparkPlan = plan.transformUp {
- case s: ShuffleExchangeLike =>
- // If an exchange follows a hash aggregate in which all functions are in partial mode,
- // then it's safe to convert the hash aggregate to flushable hash aggregate.
- val out = s.withNewChildren(
- List(
- replaceEligibleAggregates(s.child) {
- agg =>
- FlushableHashAggregateExecTransformer(
- agg.requiredChildDistributionExpressions,
- agg.groupingExpressions,
- agg.aggregateExpressions,
- agg.aggregateAttributes,
- agg.initialInputBufferOffset,
- agg.resultExpressions,
- agg.child
- )
- }
+ override def apply(plan: SparkPlan): SparkPlan = {
+ if (!GlutenConfig.getConf.enableVeloxFlushablePartialAggregation) {
+ return plan
+ }
+ plan.transformUp {
+ case s: ShuffleExchangeLike =>
+ // If an exchange follows a hash aggregate in which all functions are in partial mode,
+ // then it's safe to convert the hash aggregate to flushable hash aggregate.
+ val out = s.withNewChildren(
+ List(
+ replaceEligibleAggregates(s.child) {
+ agg =>
+ FlushableHashAggregateExecTransformer(
+ agg.requiredChildDistributionExpressions,
+ agg.groupingExpressions,
+ agg.aggregateExpressions,
+ agg.aggregateAttributes,
+ agg.initialInputBufferOffset,
+ agg.resultExpressions,
+ agg.child
+ )
+ }
+ )
)
- )
- out
+ out
+ }
}
private def replaceEligibleAggregates(plan: SparkPlan)(
diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version
index d41875c54d7d..b88675c4a96d 100644
--- a/cpp-ch/clickhouse.version
+++ b/cpp-ch/clickhouse.version
@@ -1,3 +1,3 @@
CH_ORG=Kyligence
-CH_BRANCH=rebase_ch/20240817
-CH_COMMIT=ed191291681
+CH_BRANCH=rebase_ch/20240820
+CH_COMMIT=b5b8245b022
diff --git a/cpp-ch/local-engine/Common/GlutenConfig.h b/cpp-ch/local-engine/Common/GlutenConfig.h
index ac82b0fff03a..02bb8a9f4c04 100644
--- a/cpp-ch/local-engine/Common/GlutenConfig.h
+++ b/cpp-ch/local-engine/Common/GlutenConfig.h
@@ -17,9 +17,9 @@
#pragma once
-#include
-#include
#include
+#include
+#include
namespace local_engine
{
@@ -104,7 +104,7 @@ struct JoinConfig
bool prefer_multi_join_on_clauses = true;
size_t multi_join_on_clauses_build_side_rows_limit = 10000000;
- static JoinConfig loadFromContext(DB::ContextPtr context)
+ static JoinConfig loadFromContext(const DB::ContextPtr & context)
{
JoinConfig config;
config.prefer_multi_join_on_clauses = context->getConfigRef().getBool(PREFER_MULTI_JOIN_ON_CLAUSES, true);
@@ -198,4 +198,3 @@ struct GlutenJobSchedulerConfig
}
};
}
-
diff --git a/cpp-ch/local-engine/Parser/JoinRelParser.cpp b/cpp-ch/local-engine/Parser/JoinRelParser.cpp
index ef19e007d439..30651aff1b84 100644
--- a/cpp-ch/local-engine/Parser/JoinRelParser.cpp
+++ b/cpp-ch/local-engine/Parser/JoinRelParser.cpp
@@ -209,7 +209,7 @@ DB::QueryPlanPtr JoinRelParser::parseJoin(const substrait::JoinRel & join, DB::Q
google::protobuf::StringValue optimization_info;
optimization_info.ParseFromString(join.advanced_extension().optimization().value());
auto join_opt_info = JoinOptimizationInfo::parse(optimization_info.value());
- LOG_ERROR(getLogger("JoinRelParser"), "optimizaiton info:{}", optimization_info.value());
+ LOG_DEBUG(getLogger("JoinRelParser"), "optimization info:{}", optimization_info.value());
auto storage_join = join_opt_info.is_broadcast ? BroadCastJoinBuilder::getJoin(join_opt_info.storage_join_key) : nullptr;
if (storage_join)
{
diff --git a/cpp-ch/local-engine/Storages/Parquet/ParquetConverter.h b/cpp-ch/local-engine/Storages/Parquet/ParquetConverter.h
index 312cea7efc0a..0ac16c11104d 100644
--- a/cpp-ch/local-engine/Storages/Parquet/ParquetConverter.h
+++ b/cpp-ch/local-engine/Storages/Parquet/ParquetConverter.h
@@ -35,12 +35,23 @@ template
struct ToParquet
{
using T = typename PhysicalType::c_type;
- T as(const DB::Field & value, const parquet::ColumnDescriptor &)
+ T as(const DB::Field & value, const parquet::ColumnDescriptor & s)
{
- if constexpr (std::is_same_v)
- return static_cast(value.safeGet());
+ if (s.logical_type()->is_decimal())
+ {
+ if constexpr (std::is_same_v)
+ {
+ const auto v = value.safeGet>();
+ return v.getValue().value;
+ }
+ if constexpr (std::is_same_v)
+ {
+ const auto v = value.safeGet>();
+ return v.getValue().value;
+ }
+ }
// parquet::BooleanType, parquet::Int64Type, parquet::FloatType, parquet::DoubleType
- return value.safeGet(); // FLOAT, DOUBLE, INT64
+ return value.safeGet(); // FLOAT, DOUBLE, INT64, Int32
}
};
@@ -57,28 +68,44 @@ struct ToParquet
}
};
+template
+parquet::FixedLenByteArray convertField(const DB::Field & value, uint8_t * buf, size_t type_length)
+{
+ assert(sizeof(T) >= type_length);
+
+ T val = value.safeGet>>().getValue().value;
+ std::reverse(reinterpret_cast(&val), reinterpret_cast(&val) + sizeof(T));
+ const int offset = sizeof(T) - type_length;
+
+ memcpy(buf, reinterpret_cast(&val) + offset, type_length);
+ return parquet::FixedLenByteArray(buf);
+}
+
template <>
struct ToParquet
{
- uint8_t buf[256];
+ uint8_t buf[16];
using T = parquet::FixedLenByteArray;
T as(const DB::Field & value, const parquet::ColumnDescriptor & descriptor)
{
- if (value.getType() != DB::Field::Types::Decimal128)
- throw DB::Exception(
- DB::ErrorCodes::LOGICAL_ERROR, "Field type '{}' for FIXED_LEN_BYTE_ARRAY is not supported", value.getTypeName());
- static_assert(sizeof(Int128) <= sizeof(buf));
- if (descriptor.type_length() > sizeof(Int128))
+ if (value.getType() == DB::Field::Types::Decimal256)
+ throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Field type '{}' is not supported", value.getTypeName());
+
+ static_assert(sizeof(Int128) == sizeof(buf));
+
+ if (descriptor.type_length() > sizeof(buf))
throw DB::Exception(
DB::ErrorCodes::LOGICAL_ERROR,
- "descriptor.type_length() = {} , which is > {}, e.g. sizeof(Int128)",
+ "descriptor.type_length() = {} , which is > {}, e.g. sizeof(buf)",
descriptor.type_length(),
- sizeof(Int128));
- Int128 val = value.safeGet>().getValue();
- std::reverse(reinterpret_cast(&val), reinterpret_cast(&val) + sizeof(val));
- const int offset = sizeof(Int128) - descriptor.type_length();
- memcpy(buf, reinterpret_cast(&val) + offset, descriptor.type_length());
- return parquet::FixedLenByteArray(buf);
+ sizeof(buf));
+
+ if (value.getType() == DB::Field::Types::Decimal32)
+ return convertField(value, buf, descriptor.type_length());
+ if (value.getType() == DB::Field::Types::Decimal64)
+ return convertField(value, buf, descriptor.type_length());
+
+ return convertField(value, buf, descriptor.type_length());
}
};
@@ -86,7 +113,7 @@ struct ToParquet
template
struct ConverterNumeric
{
- using From = typename Col::Container::value_type;
+ using From = typename Col::ValueType;
using To = typename DType::c_type;
const Col & column;
@@ -119,6 +146,7 @@ using ConverterInt64 = ConverterNumeric>;
using ConverterDouble = ConverterNumeric>;
+using ConverterFloat = ConverterNumeric>;
struct ConverterString
{
@@ -141,7 +169,7 @@ struct ConverterString
/// Like ConverterNumberAsFixedString, but converts to big-endian. Because that's the byte order
/// Parquet uses for decimal types and literally nothing else, for some reason.
-template
+template
struct ConverterDecimal
{
const parquet::ColumnDescriptor & descriptor;
@@ -165,7 +193,7 @@ struct ConverterDecimal
data_buf.resize(count * sizeof(T));
ptr_buf.resize(count);
memcpy(data_buf.data(), reinterpret_cast(column.getData().data() + offset), count * sizeof(T));
- const size_t offset_in_buf = sizeof(Int128) - descriptor.type_length();
+ const size_t offset_in_buf = sizeof(T) - descriptor.type_length();
;
for (size_t i = 0; i < count; ++i)
{
@@ -176,6 +204,13 @@ struct ConverterDecimal
}
};
+using Decimal128ToFLB = ConverterDecimal;
+using Decimal64ToFLB = ConverterDecimal;
+using Decimal32ToFLB = ConverterDecimal;
+
+using ConverterDecimal32 = ConverterNumeric>;
+using ConverterDecimal64 = ConverterNumeric>;
+
class BaseConverter
{
public:
@@ -239,6 +274,8 @@ std::shared_ptr> ParquetConverter::Make(const DB:
case TypeIndex::UInt32:
result = std::make_shared>(ConverterInt32_u(c));
break;
+ case TypeIndex::Decimal32:
+ result = std::make_shared>(ConverterDecimal32(c));
default:
break;
}
@@ -251,6 +288,8 @@ std::shared_ptr> ParquetConverter::Make(const DB:
case TypeIndex::UInt64:
result = std::make_shared>(ConverterInt64_u(c));
break;
+ case TypeIndex::Decimal64:
+ result = std::make_shared>(ConverterDecimal64(c));
default:
break;
}
@@ -258,6 +297,14 @@ std::shared_ptr> ParquetConverter::Make(const DB:
case parquet::Type::INT96:
break;
case parquet::Type::FLOAT:
+ switch (c->getDataType())
+ {
+ case TypeIndex::Float32:
+ result = std::make_shared>(ConverterFloat(c));
+ break;
+ default:
+ break;
+ }
break;
case parquet::Type::DOUBLE:
switch (c->getDataType())
@@ -283,8 +330,13 @@ std::shared_ptr> ParquetConverter::Make(const DB:
switch (c->getDataType())
{
case TypeIndex::Decimal128:
- result = std::make_shared>>(
- ConverterDecimal(c, desc));
+ result = std::make_shared>(Decimal128ToFLB(c, desc));
+ break;
+ case TypeIndex::Decimal64:
+ result = std::make_shared>(Decimal64ToFLB(c, desc));
+ break;
+ case TypeIndex::Decimal32:
+ result = std::make_shared>(Decimal32ToFLB(c, desc));
break;
default:
break;
diff --git a/cpp-ch/local-engine/tests/decmial_filter_push_down/18_2.json b/cpp-ch/local-engine/tests/decmial_filter_push_down/18_2.json
new file mode 100644
index 000000000000..5ad0a62325de
--- /dev/null
+++ b/cpp-ch/local-engine/tests/decmial_filter_push_down/18_2.json
@@ -0,0 +1,160 @@
+{
+ "relations": [
+ {
+ "root": {
+ "input": {
+ "filter": {
+ "common": {
+ "direct": {}
+ },
+ "input": {
+ "read": {
+ "common": {
+ "direct": {}
+ },
+ "baseSchema": {
+ "names": [
+ "a"
+ ],
+ "struct": {
+ "types": [
+ {
+ "decimal": {
+ "scale": 2,
+ "precision": 18,
+ "nullability": "NULLABILITY_NULLABLE"
+ }
+ }
+ ]
+ },
+ "columnTypes": [
+ "NORMAL_COL"
+ ]
+ },
+ "filter": {
+ "singularOrList": {
+ "value": {
+ "selection": {
+ "directReference": {
+ "structField": {}
+ }
+ }
+ },
+ "options": [
+ {
+ "literal": {
+ "decimal": {
+ "value": "yAAAAAAAAAAAAAAAAAAAAA==",
+ "precision": 18,
+ "scale": 2
+ }
+ }
+ },
+ {
+ "literal": {
+ "decimal": {
+ "value": "LAEAAAAAAAAAAAAAAAAAAA==",
+ "precision": 18,
+ "scale": 2
+ }
+ }
+ },
+ {
+ "literal": {
+ "decimal": {
+ "value": "kAEAAAAAAAAAAAAAAAAAAA==",
+ "precision": 18,
+ "scale": 2
+ }
+ }
+ },
+ {
+ "literal": {
+ "decimal": {
+ "value": "9AEAAAAAAAAAAAAAAAAAAA==",
+ "precision": 18,
+ "scale": 2
+ }
+ }
+ }
+ ]
+ }
+ },
+ "advancedExtension": {
+ "optimization": {
+ "@type": "type.googleapis.com/google.protobuf.StringValue",
+ "value": "isMergeTree=0\n"
+ }
+ }
+ }
+ },
+ "condition": {
+ "singularOrList": {
+ "value": {
+ "selection": {
+ "directReference": {
+ "structField": {}
+ }
+ }
+ },
+ "options": [
+ {
+ "literal": {
+ "decimal": {
+ "value": "yAAAAAAAAAAAAAAAAAAAAA==",
+ "precision": 18,
+ "scale": 2
+ }
+ }
+ },
+ {
+ "literal": {
+ "decimal": {
+ "value": "LAEAAAAAAAAAAAAAAAAAAA==",
+ "precision": 18,
+ "scale": 2
+ }
+ }
+ },
+ {
+ "literal": {
+ "decimal": {
+ "value": "kAEAAAAAAAAAAAAAAAAAAA==",
+ "precision": 18,
+ "scale": 2
+ }
+ }
+ },
+ {
+ "literal": {
+ "decimal": {
+ "value": "9AEAAAAAAAAAAAAAAAAAAA==",
+ "precision": 18,
+ "scale": 2
+ }
+ }
+ }
+ ]
+ }
+ }
+ }
+ },
+ "names": [
+ "a#4772"
+ ],
+ "outputSchema": {
+ "types": [
+ {
+ "decimal": {
+ "scale": 2,
+ "precision": 18,
+ "nullability": "NULLABILITY_NULLABLE"
+ }
+ }
+ ],
+ "nullability": "NULLABILITY_REQUIRED"
+ }
+ }
+ }
+ ]
+}
\ No newline at end of file
diff --git a/cpp-ch/local-engine/tests/decmial_filter_push_down/18_2_flba.snappy.parquet b/cpp-ch/local-engine/tests/decmial_filter_push_down/18_2_flba.snappy.parquet
new file mode 100644
index 000000000000..ac0b015900df
Binary files /dev/null and b/cpp-ch/local-engine/tests/decmial_filter_push_down/18_2_flba.snappy.parquet differ
diff --git a/cpp-ch/local-engine/tests/gtest_parquet_columnindex_bug.cpp b/cpp-ch/local-engine/tests/gtest_parquet_columnindex_bug.cpp
new file mode 100644
index 000000000000..ee6e70305b27
--- /dev/null
+++ b/cpp-ch/local-engine/tests/gtest_parquet_columnindex_bug.cpp
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+
+
+using namespace local_engine;
+
+using namespace DB;
+
+INCBIN(resource_embedded_pr_18_2_json, SOURCE_DIR "/utils/extern-local-engine/tests/decmial_filter_push_down/18_2.json");
+TEST(ColumnIndex, Deciaml182)
+{
+ // [precision,scale] = [18,2]
+ const auto context1 = DB::Context::createCopy(SerializedPlanParser::global_context);
+
+ auto config = ExecutorConfig::loadFromContext(context1);
+ EXPECT_TRUE(config.use_local_format) << "gtest need set use_local_format to true";
+
+ const std::string split_template
+ = R"({"items":[{"uriFile":"{replace_local_files}","partitionIndex":"0","length":"488","parquet":{},"schema":{},"metadataColumns":[{}]}]})";
+ const std::string split = replaceLocalFilesWildcards(
+ split_template, GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/decmial_filter_push_down/18_2_flba.snappy.parquet"));
+
+ SerializedPlanParser parser(context1);
+ parser.addSplitInfo(local_engine::JsonStringToBinary(split));
+
+ const auto plan = local_engine::JsonStringToMessage(
+ {reinterpret_cast(gresource_embedded_pr_18_2_jsonData), gresource_embedded_pr_18_2_jsonSize});
+
+ auto local_executor = parser.createExecutor(plan);
+ EXPECT_TRUE(local_executor->hasNext());
+ const Block & x = *local_executor->nextColumnar();
+ debug::headBlock(x);
+}
\ No newline at end of file
diff --git a/cpp-ch/local-engine/tests/json/gtest_local_engine_config.json b/cpp-ch/local-engine/tests/json/gtest_local_engine_config.json
index 10f0ea3dfdad..8ada07819bb6 100644
--- a/cpp-ch/local-engine/tests/json/gtest_local_engine_config.json
+++ b/cpp-ch/local-engine/tests/json/gtest_local_engine_config.json
@@ -260,6 +260,14 @@
"value": {
"string": "false"
}
+ },
+ {
+ "key": {
+ "string": "spark.gluten.sql.columnar.backend.ch.runtime_config.use_local_format"
+ },
+ "value": {
+ "string": "true"
+ }
}
]
}
diff --git a/cpp/velox/compute/VeloxBackend.cc b/cpp/velox/compute/VeloxBackend.cc
index cba573ade613..8dc3ade80dec 100644
--- a/cpp/velox/compute/VeloxBackend.cc
+++ b/cpp/velox/compute/VeloxBackend.cc
@@ -45,6 +45,7 @@
DECLARE_bool(velox_exception_user_stacktrace_enabled);
DECLARE_int32(velox_memory_num_shared_leaf_pools);
DECLARE_bool(velox_memory_use_hugepages);
+DECLARE_bool(velox_memory_pool_capacity_transfer_across_tasks);
DECLARE_int32(cache_prefetch_min_pct);
DECLARE_int32(gluten_velox_aysnc_timeout_on_task_stopping);
@@ -87,6 +88,9 @@ void VeloxBackend::init(const std::unordered_map& conf
FLAGS_logtostderr = true;
google::InitGoogleLogging("gluten");
+ // Allow growing buffer in another task through its memory pool.
+ FLAGS_velox_memory_pool_capacity_transfer_across_tasks = true;
+
// Avoid creating too many shared leaf pools.
FLAGS_velox_memory_num_shared_leaf_pools = 0;
diff --git a/dev/ci-velox-buildshared-centos-8.sh b/dev/ci-velox-buildshared-centos-8.sh
index 362900bd009a..f337185de379 100755
--- a/dev/ci-velox-buildshared-centos-8.sh
+++ b/dev/ci-velox-buildshared-centos-8.sh
@@ -10,9 +10,8 @@ yum install sudo patch java-1.8.0-openjdk-devel wget -y
wget https://downloads.apache.org/maven/maven-3/3.8.8/binaries/apache-maven-3.8.8-bin.tar.gz
tar -xvf apache-maven-3.8.8-bin.tar.gz && mv apache-maven-3.8.8 /usr/lib/maven
export PATH="${PATH}:/usr/lib/maven/bin"
+echo "PATH=${PATH}:/usr/lib/maven/bin" >> $GITHUB_ENV
source /opt/rh/gcc-toolset-9/enable
./dev/builddeps-veloxbe.sh --run_setup_script=OFF --enable_ep_cache=OFF --build_tests=ON \
--build_examples=ON --build_benchmarks=ON --build_protobuf=ON
-
-cd ./cpp/build && ctest -V
diff --git a/dev/ci-velox-buildstatic-centos-7.sh b/dev/ci-velox-buildstatic-centos-7.sh
index d83d443fc489..3272de95d910 100755
--- a/dev/ci-velox-buildstatic-centos-7.sh
+++ b/dev/ci-velox-buildstatic-centos-7.sh
@@ -3,8 +3,6 @@
set -e
source /opt/rh/devtoolset-9/enable
-cd $GITHUB_WORKSPACE/
-source ./dev/vcpkg/env.sh
export NUM_THREADS=4
-./dev/builddeps-veloxbe.sh --build_tests=OFF --build_benchmarks=OFF --build_arrow=OFF --enable_s3=ON \
- --enable_gcs=ON --enable_hdfs=ON --enable_abfs=ON
+./dev/builddeps-veloxbe.sh --enable_vcpkg=ON --build_arrow=OFF --build_tests=OFF --build_benchmarks=OFF \
+ --build_examples=OFF --enable_s3=ON --enable_gcs=ON --enable_hdfs=ON --enable_abfs=ON
diff --git a/dev/vcpkg/ports/simdjson/vcpkg.json b/dev/vcpkg/ports/simdjson/vcpkg.json
index 6e46382e42f8..2b74be554cfc 100644
--- a/dev/vcpkg/ports/simdjson/vcpkg.json
+++ b/dev/vcpkg/ports/simdjson/vcpkg.json
@@ -17,8 +17,7 @@
"default-features": [
"deprecated",
"exceptions",
- "threads",
- "utf8-validation"
+ "threads"
],
"features": {
"deprecated": {
diff --git a/docs/developers/UsingGperftoolsInCH.md b/docs/developers/UsingGperftoolsInCH.md
index f0d5c720b30b..5a4bbea3fbbc 100644
--- a/docs/developers/UsingGperftoolsInCH.md
+++ b/docs/developers/UsingGperftoolsInCH.md
@@ -1,3 +1,10 @@
+---
+layout: page
+title: Debug for CH Backend with gpertools Tool
+nav_order: 11
+has_children: true
+parent: /developer-overview/
+---
We need using gpertools to find the memory or CPU issue. That's what this document is about.
## Install gperftools
diff --git a/docs/developers/UsingJemallocWithCH.md b/docs/developers/UsingJemallocWithCH.md
index 626f7522d7c8..365a35dd39fe 100644
--- a/docs/developers/UsingJemallocWithCH.md
+++ b/docs/developers/UsingJemallocWithCH.md
@@ -1,3 +1,10 @@
+---
+layout: page
+title: Use Jemalloc for CH Backend
+nav_order: 12
+has_children: true
+parent: /developer-overview/
+---
We need using jemalloc to find the memory issue. That's what this document is about.
## Change code of jemalloc
diff --git a/docs/developers/VeloxUDF.md b/docs/developers/VeloxUDF.md
index 25b896929a43..6872f2d0c841 100644
--- a/docs/developers/VeloxUDF.md
+++ b/docs/developers/VeloxUDF.md
@@ -1,3 +1,10 @@
+---
+layout: page
+title: Velox UDF and UDAF
+nav_order: 13
+has_children: true
+parent: /developer-overview/
+---
# Velox User-Defined Functions (UDF) and User-Defined Aggregate Functions (UDAF)
## Introduction
diff --git a/docs/developers/velox-backend-CI.md b/docs/developers/velox-backend-CI.md
new file mode 100644
index 000000000000..e4dceffdf72e
--- /dev/null
+++ b/docs/developers/velox-backend-CI.md
@@ -0,0 +1,24 @@
+---
+layout: page
+title: Velox Backend CI
+nav_order: 14
+parent: Developer Overview
+---
+# Velox Backend CI
+
+GHA workflows are defined under `.github/workflows/`.
+
+## Docker Build
+We have a weekly job to build a docker based on `Dockerfile.gha` for CI verification, defined in `docker_image.yml`.
+
+## Vcpkg Caching
+Gluten main branch is pulled down during docker build. And vcpkg will cache binary data of all dependencies defined under dev/vcpkg.
+These binary data is cached into `/var/cache/vcpkg` and CI job can re-use them in new build. By setting `VCPKG_BINARY_SOURCES=clear` in env.,
+reusing vcpkg cache can be disabled.
+
+## Arrow Libs Pre-installation
+Arrow libs are pre-installed in docker, assuming they are not actively changed, then not necessarily to be re-built every time.
+
+## Updating Docker Image
+Two GitHub secrets `DOCKERHUB_USER` & `DOCKERHUB_TOKEN` can be used to push docker image to docker hub: https://hub.docker.com/r/apache/gluten/tags.
+Note GitHub secrets are not retrievable in PR from forked repo.
\ No newline at end of file
diff --git a/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala b/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala
index dbf927909187..6e3484dfa969 100644
--- a/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala
+++ b/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala
@@ -17,12 +17,11 @@
package org.apache.gluten
import org.apache.gluten.GlutenConfig.GLUTEN_DEFAULT_SESSION_TIMEZONE_KEY
-import org.apache.gluten.GlutenPlugin.{GLUTEN_SESSION_EXTENSION_NAME, SPARK_SESSION_EXTS_KEY}
import org.apache.gluten.backendsapi.BackendsApiManager
import org.apache.gluten.events.GlutenBuildInfoEvent
import org.apache.gluten.exception.GlutenException
import org.apache.gluten.expression.ExpressionMappings
-import org.apache.gluten.extension.{ColumnarOverrides, OthersExtensionOverrides, QueryStagePrepOverrides}
+import org.apache.gluten.extension.GlutenSessionExtensions.{GLUTEN_SESSION_EXTENSION_NAME, SPARK_SESSION_EXTS_KEY}
import org.apache.gluten.test.TestStats
import org.apache.gluten.utils.TaskListener
@@ -31,14 +30,13 @@ import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext,
import org.apache.spark.internal.Logging
import org.apache.spark.listener.GlutenListenerFactory
import org.apache.spark.network.util.JavaUtils
-import org.apache.spark.sql.SparkSessionExtensions
import org.apache.spark.sql.execution.ui.GlutenEventUtils
-import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.utils.ExpressionUtil
import org.apache.spark.util.{SparkResourceUtil, TaskResources}
import java.util
-import java.util.{Collections, Objects}
+import java.util.Collections
import scala.collection.mutable
@@ -298,25 +296,4 @@ private[gluten] class GlutenExecutorPlugin extends ExecutorPlugin {
}
}
-private[gluten] class GlutenSessionExtensions extends (SparkSessionExtensions => Unit) {
- override def apply(exts: SparkSessionExtensions): Unit = {
- GlutenPlugin.DEFAULT_INJECTORS.foreach(injector => injector.inject(exts))
- }
-}
-
-private[gluten] trait GlutenSparkExtensionsInjector {
- def inject(extensions: SparkSessionExtensions): Unit
-}
-
-private[gluten] object GlutenPlugin {
- val SPARK_SESSION_EXTS_KEY: String = StaticSQLConf.SPARK_SESSION_EXTENSIONS.key
- val GLUTEN_SESSION_EXTENSION_NAME: String =
- Objects.requireNonNull(classOf[GlutenSessionExtensions].getCanonicalName)
-
- /** Specify all injectors that Gluten is using in following list. */
- val DEFAULT_INJECTORS: List[GlutenSparkExtensionsInjector] = List(
- QueryStagePrepOverrides,
- ColumnarOverrides,
- OthersExtensionOverrides
- )
-}
+private object GlutenPlugin {}
diff --git a/gluten-core/src/main/scala/org/apache/gluten/backendsapi/Backend.scala b/gluten-core/src/main/scala/org/apache/gluten/backendsapi/Backend.scala
index 2c465ac61993..3a597552207b 100644
--- a/gluten-core/src/main/scala/org/apache/gluten/backendsapi/Backend.scala
+++ b/gluten-core/src/main/scala/org/apache/gluten/backendsapi/Backend.scala
@@ -33,6 +33,8 @@ trait Backend {
def listenerApi(): ListenerApi
+ def ruleApi(): RuleApi
+
def settings(): BackendSettingsApi
}
diff --git a/gluten-core/src/main/scala/org/apache/gluten/backendsapi/BackendsApiManager.scala b/gluten-core/src/main/scala/org/apache/gluten/backendsapi/BackendsApiManager.scala
index f2c93d8c70fc..16aa9161eba0 100644
--- a/gluten-core/src/main/scala/org/apache/gluten/backendsapi/BackendsApiManager.scala
+++ b/gluten-core/src/main/scala/org/apache/gluten/backendsapi/BackendsApiManager.scala
@@ -83,6 +83,10 @@ object BackendsApiManager {
backend.metricsApi()
}
+ def getRuleApiInstance: RuleApi = {
+ backend.ruleApi()
+ }
+
def getSettings: BackendSettingsApi = {
backend.settings
}
diff --git a/gluten-core/src/main/scala/org/apache/gluten/backendsapi/RuleApi.scala b/gluten-core/src/main/scala/org/apache/gluten/backendsapi/RuleApi.scala
new file mode 100644
index 000000000000..f8669a6fe049
--- /dev/null
+++ b/gluten-core/src/main/scala/org/apache/gluten/backendsapi/RuleApi.scala
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.backendsapi
+
+import org.apache.gluten.extension.injector.RuleInjector
+
+trait RuleApi {
+ // Injects all Gluten / Spark query planner rules used by the backend.
+ def injectRules(injector: RuleInjector): Unit
+}
diff --git a/gluten-core/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala b/gluten-core/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala
index 3b9e87a2055a..0227ed5da127 100644
--- a/gluten-core/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala
+++ b/gluten-core/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala
@@ -27,20 +27,14 @@ import org.apache.spark.ShuffleDependency
import org.apache.spark.rdd.RDD
import org.apache.spark.serializer.Serializer
import org.apache.spark.shuffle.{GenShuffleWriterParameters, GlutenShuffleWriterWrapper}
-import org.apache.spark.sql.{SparkSession, Strategy}
-import org.apache.spark.sql.catalyst.FunctionIdentifier
-import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder
import org.apache.spark.sql.catalyst.catalog.BucketSpec
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.optimizer.BuildSide
-import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.plans.JoinType
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning}
-import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.execution.{ColumnarWriteFilesExec, FileSourceScanExec, GenerateExec, LeafExecNode, SparkPlan}
+import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.datasources.FileFormat
import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, FileScan}
import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
@@ -392,69 +386,6 @@ trait SparkPlanExecApi {
child: SparkPlan,
evalType: Int): SparkPlan
- /**
- * Generate extended DataSourceV2 Strategies. Currently only for ClickHouse backend.
- *
- * @return
- */
- def genExtendedDataSourceV2Strategies(): List[SparkSession => Strategy]
-
- /**
- * Generate extended query stage preparation rules.
- *
- * @return
- */
- def genExtendedQueryStagePrepRules(): List[SparkSession => Rule[SparkPlan]]
-
- /**
- * Generate extended Analyzers. Currently only for ClickHouse backend.
- *
- * @return
- */
- def genExtendedAnalyzers(): List[SparkSession => Rule[LogicalPlan]]
-
- /**
- * Generate extended Optimizers. Currently only for Velox backend.
- *
- * @return
- */
- def genExtendedOptimizers(): List[SparkSession => Rule[LogicalPlan]]
-
- /**
- * Generate extended Strategies
- *
- * @return
- */
- def genExtendedStrategies(): List[SparkSession => Strategy]
-
- /**
- * Generate extended columnar pre-rules, in the validation phase.
- *
- * @return
- */
- def genExtendedColumnarValidationRules(): List[SparkSession => Rule[SparkPlan]]
-
- /**
- * Generate extended columnar transform-rules.
- *
- * @return
- */
- def genExtendedColumnarTransformRules(): List[SparkSession => Rule[SparkPlan]]
-
- /**
- * Generate extended columnar post-rules.
- *
- * @return
- */
- def genExtendedColumnarPostRules(): List[SparkSession => Rule[SparkPlan]] = {
- SparkShimLoader.getSparkShims.getExtendedColumnarPostRules() ::: List()
- }
-
- def genInjectPostHocResolutionRules(): List[SparkSession => Rule[LogicalPlan]]
-
- def genInjectExtendedParser(): List[(SparkSession, ParserInterface) => ParserInterface] =
- List.empty
-
def genGetStructFieldTransformer(
substraitExprName: String,
childTransformer: ExpressionTransformer,
@@ -665,8 +596,6 @@ trait SparkPlanExecApi {
}
}
- def genInjectedFunctions(): Seq[(FunctionIdentifier, ExpressionInfo, FunctionBuilder)] = Seq.empty
-
def rewriteSpillPath(path: String): String = path
/**
diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/ColumnarOverrides.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/ColumnarOverrides.scala
index 067976b63b2c..c5a9afec3210 100644
--- a/gluten-core/src/main/scala/org/apache/gluten/extension/ColumnarOverrides.scala
+++ b/gluten-core/src/main/scala/org/apache/gluten/extension/ColumnarOverrides.scala
@@ -16,17 +16,14 @@
*/
package org.apache.gluten.extension
-import org.apache.gluten.{GlutenConfig, GlutenSparkExtensionsInjector}
import org.apache.gluten.extension.columnar._
-import org.apache.gluten.extension.columnar.enumerated.EnumeratedApplier
-import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier
import org.apache.gluten.extension.columnar.transition.Transitions
import org.apache.gluten.utils.LogLevelUtil
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.{SparkSession, SparkSessionExtensions}
+import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.rules.Rule
@@ -95,7 +92,9 @@ object ColumnarOverrideRules {
}
}
-case class ColumnarOverrideRules(session: SparkSession)
+case class ColumnarOverrideRules(
+ session: SparkSession,
+ applierBuilder: SparkSession => ColumnarRuleApplier)
extends ColumnarRule
with Logging
with LogLevelUtil {
@@ -117,19 +116,11 @@ case class ColumnarOverrideRules(session: SparkSession)
val outputsColumnar = OutputsColumnarTester.inferOutputsColumnar(plan)
val unwrapped = OutputsColumnarTester.unwrap(plan)
val vanillaPlan = Transitions.insertTransitions(unwrapped, outputsColumnar)
- val applier: ColumnarRuleApplier = if (GlutenConfig.getConf.enableRas) {
- new EnumeratedApplier(session)
- } else {
- new HeuristicApplier(session)
- }
+ val applier = applierBuilder.apply(session)
val out = applier.apply(vanillaPlan, outputsColumnar)
out
}
}
-object ColumnarOverrides extends GlutenSparkExtensionsInjector {
- override def inject(extensions: SparkSessionExtensions): Unit = {
- extensions.injectColumnar(spark => ColumnarOverrideRules(spark))
- }
-}
+object ColumnarOverrides {}
diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala
new file mode 100644
index 000000000000..4456dda61528
--- /dev/null
+++ b/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.extension
+
+import org.apache.gluten.backendsapi.BackendsApiManager
+import org.apache.gluten.extension.injector.RuleInjector
+
+import org.apache.spark.sql.SparkSessionExtensions
+import org.apache.spark.sql.internal.StaticSQLConf
+
+import java.util.Objects
+
+private[gluten] class GlutenSessionExtensions extends (SparkSessionExtensions => Unit) {
+ override def apply(exts: SparkSessionExtensions): Unit = {
+ val injector = new RuleInjector()
+ BackendsApiManager.getRuleApiInstance.injectRules(injector)
+ injector.inject(exts)
+ }
+}
+
+private[gluten] object GlutenSessionExtensions {
+ val SPARK_SESSION_EXTS_KEY: String = StaticSQLConf.SPARK_SESSION_EXTENSIONS.key
+ val GLUTEN_SESSION_EXTENSION_NAME: String =
+ Objects.requireNonNull(classOf[GlutenSessionExtensions].getCanonicalName)
+}
diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/OthersExtensionOverrides.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/OthersExtensionOverrides.scala
deleted file mode 100644
index f2ccf6e81ca1..000000000000
--- a/gluten-core/src/main/scala/org/apache/gluten/extension/OthersExtensionOverrides.scala
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.
- */
-package org.apache.gluten.extension
-
-import org.apache.gluten.GlutenSparkExtensionsInjector
-import org.apache.gluten.backendsapi.BackendsApiManager
-
-import org.apache.spark.sql.SparkSessionExtensions
-
-object OthersExtensionOverrides extends GlutenSparkExtensionsInjector {
- override def inject(extensions: SparkSessionExtensions): Unit = {
- BackendsApiManager.getSparkPlanExecApiInstance
- .genInjectExtendedParser()
- .foreach(extensions.injectParser)
- BackendsApiManager.getSparkPlanExecApiInstance
- .genExtendedAnalyzers()
- .foreach(extensions.injectResolutionRule)
- BackendsApiManager.getSparkPlanExecApiInstance
- .genExtendedOptimizers()
- .foreach(extensions.injectOptimizerRule)
- BackendsApiManager.getSparkPlanExecApiInstance
- .genExtendedDataSourceV2Strategies()
- .foreach(extensions.injectPlannerStrategy)
- BackendsApiManager.getSparkPlanExecApiInstance
- .genExtendedStrategies()
- .foreach(extensions.injectPlannerStrategy)
- BackendsApiManager.getSparkPlanExecApiInstance
- .genInjectedFunctions()
- .foreach(extensions.injectFunction)
- BackendsApiManager.getSparkPlanExecApiInstance
- .genInjectPostHocResolutionRules()
- .foreach(extensions.injectPostHocResolutionRule)
- }
-}
diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/QueryStagePrepOverrides.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/QueryStagePrepOverrides.scala
deleted file mode 100644
index 8f9e2326ca71..000000000000
--- a/gluten-core/src/main/scala/org/apache/gluten/extension/QueryStagePrepOverrides.scala
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.
- */
-package org.apache.gluten.extension
-
-import org.apache.gluten.GlutenSparkExtensionsInjector
-import org.apache.gluten.backendsapi.BackendsApiManager
-
-import org.apache.spark.sql.{SparkSession, SparkSessionExtensions}
-import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.execution.SparkPlan
-
-/*
- * 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.
- */
-
-object QueryStagePrepOverrides extends GlutenSparkExtensionsInjector {
- private val RULES: Seq[SparkSession => Rule[SparkPlan]] =
- BackendsApiManager.getSparkPlanExecApiInstance.genExtendedQueryStagePrepRules()
-
- override def inject(extensions: SparkSessionExtensions): Unit = {
- RULES.foreach(extensions.injectQueryStagePrepRule)
- }
-}
diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/ColumnarRuleApplier.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/ColumnarRuleApplier.scala
index 27213698b9f2..9b78ccd11de2 100644
--- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/ColumnarRuleApplier.scala
+++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/ColumnarRuleApplier.scala
@@ -17,10 +17,12 @@
package org.apache.gluten.extension.columnar
import org.apache.gluten.GlutenConfig
+import org.apache.gluten.extension.columnar.util.AdaptiveContext
import org.apache.gluten.metrics.GlutenTimeMetric
import org.apache.gluten.utils.LogLevelUtil
import org.apache.spark.internal.Logging
+import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor}
import org.apache.spark.sql.catalyst.util.sideBySide
import org.apache.spark.sql.execution.SparkPlan
@@ -30,6 +32,17 @@ trait ColumnarRuleApplier {
}
object ColumnarRuleApplier {
+ type ColumnarRuleBuilder = ColumnarRuleCall => Rule[SparkPlan]
+
+ class ColumnarRuleCall(
+ val session: SparkSession,
+ val ac: AdaptiveContext,
+ val outputsColumnar: Boolean) {
+ val conf: GlutenConfig = {
+ new GlutenConfig(session.sessionState.conf)
+ }
+ }
+
class Executor(phase: String, rules: Seq[Rule[SparkPlan]]) extends RuleExecutor[SparkPlan] {
private val batch: Batch =
Batch(s"Columnar (Phase [$phase])", Once, rules.map(r => new LoggedRule(r)): _*)
diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala
index 5cf3961c548b..bebce3a61ae8 100644
--- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala
+++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala
@@ -16,11 +16,8 @@
*/
package org.apache.gluten.extension.columnar.enumerated
-import org.apache.gluten.GlutenConfig
-import org.apache.gluten.backendsapi.BackendsApiManager
import org.apache.gluten.extension.columnar._
-import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast}
-import org.apache.gluten.extension.columnar.transition.{InsertTransitions, RemoveTransitions}
+import org.apache.gluten.extension.columnar.ColumnarRuleApplier.{ColumnarRuleBuilder, ColumnarRuleCall}
import org.apache.gluten.extension.columnar.util.AdaptiveContext
import org.apache.gluten.utils.{LogLevelUtil, PhysicalPlanSelector}
@@ -28,8 +25,7 @@ import org.apache.spark.annotation.Experimental
import org.apache.spark.internal.Logging
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, GlutenFallbackReporter, SparkPlan}
-import org.apache.spark.util.SparkRuleUtil
+import org.apache.spark.sql.execution.SparkPlan
/**
* Columnar rule applier that optimizes, implements Spark plan into Gluten plan by enumerating on
@@ -40,7 +36,7 @@ import org.apache.spark.util.SparkRuleUtil
* implementing them in EnumeratedTransform.
*/
@Experimental
-class EnumeratedApplier(session: SparkSession)
+class EnumeratedApplier(session: SparkSession, ruleBuilders: Seq[ColumnarRuleBuilder])
extends ColumnarRuleApplier
with Logging
with LogLevelUtil {
@@ -53,22 +49,18 @@ class EnumeratedApplier(session: SparkSession)
}
private val adaptiveContext = AdaptiveContext(session, aqeStackTraceIndex)
- override def apply(plan: SparkPlan, outputsColumnar: Boolean): SparkPlan =
+ override def apply(plan: SparkPlan, outputsColumnar: Boolean): SparkPlan = {
+ val call = new ColumnarRuleCall(session, adaptiveContext, outputsColumnar)
PhysicalPlanSelector.maybe(session, plan) {
- val transformed =
- transformPlan("transform", transformRules(outputsColumnar).map(_(session)), plan)
- val postPlan = maybeAqe {
- transformPlan("post", postRules().map(_(session)), transformed)
+ val finalPlan = maybeAqe {
+ apply0(ruleBuilders.map(b => b(call)), plan)
}
- val finalPlan = transformPlan("final", finalRules().map(_(session)), postPlan)
finalPlan
}
+ }
- private def transformPlan(
- phase: String,
- rules: Seq[Rule[SparkPlan]],
- plan: SparkPlan): SparkPlan = {
- val executor = new ColumnarRuleApplier.Executor(phase, rules)
+ private def apply0(rules: Seq[Rule[SparkPlan]], plan: SparkPlan): SparkPlan = {
+ val executor = new ColumnarRuleApplier.Executor("ras", rules)
executor.execute(plan)
}
@@ -80,61 +72,4 @@ class EnumeratedApplier(session: SparkSession)
adaptiveContext.resetAdaptiveContext()
}
}
-
- /**
- * Rules to let planner create a suggested Gluten plan being sent to `fallbackPolicies` in which
- * the plan will be breakdown and decided to be fallen back or not.
- */
- private def transformRules(outputsColumnar: Boolean): Seq[SparkSession => Rule[SparkPlan]] = {
- List(
- (_: SparkSession) => RemoveTransitions,
- (spark: SparkSession) => FallbackOnANSIMode(spark),
- (spark: SparkSession) => PlanOneRowRelation(spark),
- (_: SparkSession) => FallbackEmptySchemaRelation(),
- (_: SparkSession) => RewriteSubqueryBroadcast()
- ) :::
- BackendsApiManager.getSparkPlanExecApiInstance.genExtendedColumnarValidationRules() :::
- List((spark: SparkSession) => MergeTwoPhasesHashBaseAggregate(spark)) :::
- List(
- (session: SparkSession) => EnumeratedTransform(session, outputsColumnar),
- (_: SparkSession) => RemoveTransitions
- ) :::
- List(
- (_: SparkSession) => RemoveNativeWriteFilesSortAndProject(),
- (spark: SparkSession) => RewriteTransformer(spark),
- (_: SparkSession) => EnsureLocalSortRequirements,
- (_: SparkSession) => EliminateLocalSort,
- (_: SparkSession) => CollapseProjectExecTransformer
- ) :::
- BackendsApiManager.getSparkPlanExecApiInstance.genExtendedColumnarTransformRules() :::
- SparkRuleUtil
- .extendedColumnarRules(session, GlutenConfig.getConf.extendedColumnarTransformRules) :::
- List((_: SparkSession) => InsertTransitions(outputsColumnar))
- }
-
- /**
- * Rules applying to non-fallen-back Gluten plans. To do some post cleanup works on the plan to
- * make sure it be able to run and be compatible with Spark's execution engine.
- */
- private def postRules(): Seq[SparkSession => Rule[SparkPlan]] =
- List(
- (s: SparkSession) => RemoveTopmostColumnarToRow(s, adaptiveContext.isAdaptiveContext())) :::
- BackendsApiManager.getSparkPlanExecApiInstance.genExtendedColumnarPostRules() :::
- List((_: SparkSession) => ColumnarCollapseTransformStages(GlutenConfig.getConf)) :::
- SparkRuleUtil.extendedColumnarRules(session, GlutenConfig.getConf.extendedColumnarPostRules)
-
- /*
- * Rules consistently applying to all input plans after all other rules have been applied, despite
- * whether the input plan is fallen back or not.
- */
- private def finalRules(): Seq[SparkSession => Rule[SparkPlan]] = {
- List(
- // The rule is required despite whether the stage is fallen back or not. Since
- // ColumnarCachedBatchSerializer is statically registered to Spark without a columnar rule
- // when columnar table cache is enabled.
- (s: SparkSession) => RemoveGlutenTableCacheColumnarToRow(s),
- (s: SparkSession) => GlutenFallbackReporter(GlutenConfig.getConf, s),
- (_: SparkSession) => RemoveFallbackTagRule()
- )
- }
}
diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicApplier.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicApplier.scala
index f776a1dcc3cd..dea9f01df2a5 100644
--- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicApplier.scala
+++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicApplier.scala
@@ -16,26 +16,26 @@
*/
package org.apache.gluten.extension.columnar.heuristic
-import org.apache.gluten.GlutenConfig
-import org.apache.gluten.backendsapi.BackendsApiManager
import org.apache.gluten.extension.columnar._
-import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast, TransformPreOverrides}
-import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager
-import org.apache.gluten.extension.columnar.transition.{InsertTransitions, RemoveTransitions}
+import org.apache.gluten.extension.columnar.ColumnarRuleApplier.{ColumnarRuleBuilder, ColumnarRuleCall}
import org.apache.gluten.extension.columnar.util.AdaptiveContext
import org.apache.gluten.utils.{LogLevelUtil, PhysicalPlanSelector}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, GlutenFallbackReporter, SparkPlan}
-import org.apache.spark.util.SparkRuleUtil
+import org.apache.spark.sql.execution.SparkPlan
/**
* Columnar rule applier that optimizes, implements Spark plan into Gluten plan by heuristically
* applying columnar rules in fixed order.
*/
-class HeuristicApplier(session: SparkSession)
+class HeuristicApplier(
+ session: SparkSession,
+ transformBuilders: Seq[ColumnarRuleBuilder],
+ fallbackPolicyBuilders: Seq[ColumnarRuleBuilder],
+ postBuilders: Seq[ColumnarRuleBuilder],
+ finalBuilders: Seq[ColumnarRuleBuilder])
extends ColumnarRuleApplier
with Logging
with LogLevelUtil {
@@ -49,27 +49,27 @@ class HeuristicApplier(session: SparkSession)
private val adaptiveContext = AdaptiveContext(session, aqeStackTraceIndex)
override def apply(plan: SparkPlan, outputsColumnar: Boolean): SparkPlan = {
- withTransformRules(transformRules(outputsColumnar)).apply(plan)
+ val call = new ColumnarRuleCall(session, adaptiveContext, outputsColumnar)
+ makeRule(call).apply(plan)
}
- // Visible for testing.
- def withTransformRules(transformRules: Seq[SparkSession => Rule[SparkPlan]]): Rule[SparkPlan] =
+ private def makeRule(call: ColumnarRuleCall): Rule[SparkPlan] =
plan =>
PhysicalPlanSelector.maybe(session, plan) {
val finalPlan = prepareFallback(plan) {
p =>
- val suggestedPlan = transformPlan("transform", transformRules.map(_(session)), p)
- transformPlan("fallback", fallbackPolicies().map(_(session)), suggestedPlan) match {
+ val suggestedPlan = transformPlan("transform", transformRules(call), p)
+ transformPlan("fallback", fallbackPolicies(call), suggestedPlan) match {
case FallbackNode(fallbackPlan) =>
// we should use vanilla c2r rather than native c2r,
// and there should be no `GlutenPlan` any more,
// so skip the `postRules()`.
fallbackPlan
case plan =>
- transformPlan("post", postRules().map(_(session)), plan)
+ transformPlan("post", postRules(call), plan)
}
}
- transformPlan("final", finalRules().map(_(session)), finalPlan)
+ transformPlan("final", finalRules(call), finalPlan)
}
private def transformPlan(
@@ -95,69 +95,32 @@ class HeuristicApplier(session: SparkSession)
* Rules to let planner create a suggested Gluten plan being sent to `fallbackPolicies` in which
* the plan will be breakdown and decided to be fallen back or not.
*/
- private def transformRules(outputsColumnar: Boolean): Seq[SparkSession => Rule[SparkPlan]] = {
- List(
- (_: SparkSession) => RemoveTransitions,
- (spark: SparkSession) => FallbackOnANSIMode(spark),
- (spark: SparkSession) => FallbackMultiCodegens(spark),
- (spark: SparkSession) => PlanOneRowRelation(spark),
- (_: SparkSession) => RewriteSubqueryBroadcast()
- ) :::
- BackendsApiManager.getSparkPlanExecApiInstance.genExtendedColumnarValidationRules() :::
- List(
- (_: SparkSession) => FallbackEmptySchemaRelation(),
- (spark: SparkSession) => MergeTwoPhasesHashBaseAggregate(spark),
- (_: SparkSession) => RewriteSparkPlanRulesManager(),
- (_: SparkSession) => AddFallbackTagRule()
- ) :::
- List((_: SparkSession) => TransformPreOverrides()) :::
- List(
- (_: SparkSession) => RemoveNativeWriteFilesSortAndProject(),
- (spark: SparkSession) => RewriteTransformer(spark),
- (_: SparkSession) => EnsureLocalSortRequirements,
- (_: SparkSession) => EliminateLocalSort,
- (_: SparkSession) => CollapseProjectExecTransformer
- ) :::
- BackendsApiManager.getSparkPlanExecApiInstance.genExtendedColumnarTransformRules() :::
- SparkRuleUtil
- .extendedColumnarRules(session, GlutenConfig.getConf.extendedColumnarTransformRules) :::
- List((_: SparkSession) => InsertTransitions(outputsColumnar))
+ private def transformRules(call: ColumnarRuleCall): Seq[Rule[SparkPlan]] = {
+ transformBuilders.map(b => b.apply(call))
}
/**
* Rules to add wrapper `FallbackNode`s on top of the input plan, as hints to make planner fall
* back the whole input plan to the original vanilla Spark plan.
*/
- private def fallbackPolicies(): Seq[SparkSession => Rule[SparkPlan]] = {
- List(
- (_: SparkSession) =>
- ExpandFallbackPolicy(adaptiveContext.isAdaptiveContext(), adaptiveContext.originalPlan()))
+ private def fallbackPolicies(call: ColumnarRuleCall): Seq[Rule[SparkPlan]] = {
+ fallbackPolicyBuilders.map(b => b.apply(call))
}
/**
* Rules applying to non-fallen-back Gluten plans. To do some post cleanup works on the plan to
* make sure it be able to run and be compatible with Spark's execution engine.
*/
- private def postRules(): Seq[SparkSession => Rule[SparkPlan]] =
- List(
- (s: SparkSession) => RemoveTopmostColumnarToRow(s, adaptiveContext.isAdaptiveContext())) :::
- BackendsApiManager.getSparkPlanExecApiInstance.genExtendedColumnarPostRules() :::
- List((_: SparkSession) => ColumnarCollapseTransformStages(GlutenConfig.getConf)) :::
- SparkRuleUtil.extendedColumnarRules(session, GlutenConfig.getConf.extendedColumnarPostRules)
+ private def postRules(call: ColumnarRuleCall): Seq[Rule[SparkPlan]] = {
+ postBuilders.map(b => b.apply(call))
+ }
/*
* Rules consistently applying to all input plans after all other rules have been applied, despite
* whether the input plan is fallen back or not.
*/
- private def finalRules(): Seq[SparkSession => Rule[SparkPlan]] = {
- List(
- // The rule is required despite whether the stage is fallen back or not. Since
- // ColumnarCachedBatchSerializer is statically registered to Spark without a columnar rule
- // when columnar table cache is enabled.
- (s: SparkSession) => RemoveGlutenTableCacheColumnarToRow(s),
- (s: SparkSession) => GlutenFallbackReporter(GlutenConfig.getConf, s),
- (_: SparkSession) => RemoveFallbackTagRule()
- )
+ private def finalRules(call: ColumnarRuleCall): Seq[Rule[SparkPlan]] = {
+ finalBuilders.map(b => b.apply(call))
}
// Just for test use.
@@ -166,3 +129,5 @@ class HeuristicApplier(session: SparkSession)
this
}
}
+
+object HeuristicApplier {}
diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/util/AdaptiveContext.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/util/AdaptiveContext.scala
index 4a9d69f8f0b1..e1f594fd36e5 100644
--- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/util/AdaptiveContext.scala
+++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/util/AdaptiveContext.scala
@@ -22,6 +22,7 @@ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
import scala.collection.mutable.ListBuffer
+// Since: https://github.com/apache/incubator-gluten/pull/3294.
sealed trait AdaptiveContext {
def enableAdaptiveContext(): Unit
def isAdaptiveContext(): Boolean
diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala
new file mode 100644
index 000000000000..728e569cc4eb
--- /dev/null
+++ b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala
@@ -0,0 +1,94 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.extension.injector
+
+import org.apache.gluten.GlutenConfig
+import org.apache.gluten.extension.ColumnarOverrideRules
+import org.apache.gluten.extension.columnar.ColumnarRuleApplier
+import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleBuilder
+import org.apache.gluten.extension.columnar.enumerated.EnumeratedApplier
+import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier
+
+import org.apache.spark.sql.{SparkSession, SparkSessionExtensions}
+
+import scala.collection.mutable
+
+/** Injector used to inject query planner rules into Gluten. */
+class GlutenInjector private[injector] {
+ import GlutenInjector._
+ val legacy: LegacyInjector = new LegacyInjector()
+ val ras: RasInjector = new RasInjector()
+
+ private[injector] def inject(extensions: SparkSessionExtensions): Unit = {
+ val ruleBuilder = (session: SparkSession) => new ColumnarOverrideRules(session, applier)
+ extensions.injectColumnar(session => ruleBuilder(session))
+ }
+
+ private def applier(session: SparkSession): ColumnarRuleApplier = {
+ val conf = new GlutenConfig(session.sessionState.conf)
+ if (conf.enableRas) {
+ return ras.createApplier(session)
+ }
+ legacy.createApplier(session)
+ }
+}
+
+object GlutenInjector {
+ class LegacyInjector {
+ private val transformBuilders = mutable.Buffer.empty[ColumnarRuleBuilder]
+ private val fallbackPolicyBuilders = mutable.Buffer.empty[ColumnarRuleBuilder]
+ private val postBuilders = mutable.Buffer.empty[ColumnarRuleBuilder]
+ private val finalBuilders = mutable.Buffer.empty[ColumnarRuleBuilder]
+
+ def injectTransform(builder: ColumnarRuleBuilder): Unit = {
+ transformBuilders += builder
+ }
+
+ def injectFallbackPolicy(builder: ColumnarRuleBuilder): Unit = {
+ fallbackPolicyBuilders += builder
+ }
+
+ def injectPost(builder: ColumnarRuleBuilder): Unit = {
+ postBuilders += builder
+ }
+
+ def injectFinal(builder: ColumnarRuleBuilder): Unit = {
+ finalBuilders += builder
+ }
+
+ private[injector] def createApplier(session: SparkSession): ColumnarRuleApplier = {
+ new HeuristicApplier(
+ session,
+ transformBuilders.toSeq,
+ fallbackPolicyBuilders.toSeq,
+ postBuilders.toSeq,
+ finalBuilders.toSeq)
+ }
+ }
+
+ class RasInjector {
+ private val ruleBuilders = mutable.Buffer.empty[ColumnarRuleBuilder]
+
+ def inject(builder: ColumnarRuleBuilder): Unit = {
+ ruleBuilders += builder
+ }
+
+ private[injector] def createApplier(session: SparkSession): ColumnarRuleApplier = {
+ new EnumeratedApplier(session, ruleBuilders.toSeq)
+ }
+ }
+}
diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/RuleInjector.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/RuleInjector.scala
new file mode 100644
index 000000000000..bccbd38b26d5
--- /dev/null
+++ b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/RuleInjector.scala
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.extension.injector
+
+import org.apache.spark.sql.SparkSessionExtensions
+
+/** Injector used to inject query planner rules into Spark and Gluten. */
+class RuleInjector {
+ val spark: SparkInjector = new SparkInjector()
+ val gluten: GlutenInjector = new GlutenInjector()
+
+ private[extension] def inject(extensions: SparkSessionExtensions): Unit = {
+ spark.inject(extensions)
+ gluten.inject(extensions)
+ }
+}
+
+object RuleInjector {}
diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/SparkInjector.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/SparkInjector.scala
new file mode 100644
index 000000000000..6935e61bdd5b
--- /dev/null
+++ b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/SparkInjector.scala
@@ -0,0 +1,83 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.extension.injector
+
+import org.apache.spark.sql.{SparkSession, SparkSessionExtensions, Strategy}
+import org.apache.spark.sql.catalyst.FunctionIdentifier
+import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder
+import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
+import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.SparkPlan
+
+import scala.collection.mutable
+
+/** Injector used to inject query planner rules into Spark. */
+class SparkInjector private[injector] {
+ private type RuleBuilder = SparkSession => Rule[LogicalPlan]
+ private type StrategyBuilder = SparkSession => Strategy
+ private type ParserBuilder = (SparkSession, ParserInterface) => ParserInterface
+ private type FunctionDescription = (FunctionIdentifier, ExpressionInfo, FunctionBuilder)
+ private type QueryStagePrepRuleBuilder = SparkSession => Rule[SparkPlan]
+
+ private val queryStagePrepRuleBuilders = mutable.Buffer.empty[QueryStagePrepRuleBuilder]
+ private val parserBuilders = mutable.Buffer.empty[ParserBuilder]
+ private val resolutionRuleBuilders = mutable.Buffer.empty[RuleBuilder]
+ private val optimizerRules = mutable.Buffer.empty[RuleBuilder]
+ private val plannerStrategyBuilders = mutable.Buffer.empty[StrategyBuilder]
+ private val injectedFunctions = mutable.Buffer.empty[FunctionDescription]
+ private val postHocResolutionRuleBuilders = mutable.Buffer.empty[RuleBuilder]
+
+ def injectQueryStagePrepRule(builder: QueryStagePrepRuleBuilder): Unit = {
+ queryStagePrepRuleBuilders += builder
+ }
+
+ def injectParser(builder: ParserBuilder): Unit = {
+ parserBuilders += builder
+ }
+
+ def injectResolutionRule(builder: RuleBuilder): Unit = {
+ resolutionRuleBuilders += builder
+ }
+
+ def injectOptimizerRule(builder: RuleBuilder): Unit = {
+ optimizerRules += builder
+ }
+
+ def injectPlannerStrategy(builder: StrategyBuilder): Unit = {
+ plannerStrategyBuilders += builder
+ }
+
+ def injectFunction(functionDescription: FunctionDescription): Unit = {
+ injectedFunctions += functionDescription
+ }
+
+ def injectPostHocResolutionRule(builder: RuleBuilder): Unit = {
+ postHocResolutionRuleBuilders += builder
+ }
+
+ private[injector] def inject(extensions: SparkSessionExtensions): Unit = {
+ queryStagePrepRuleBuilders.foreach(extensions.injectQueryStagePrepRule)
+ parserBuilders.foreach(extensions.injectParser)
+ resolutionRuleBuilders.foreach(extensions.injectResolutionRule)
+ optimizerRules.foreach(extensions.injectOptimizerRule)
+ plannerStrategyBuilders.foreach(extensions.injectPlannerStrategy)
+ injectedFunctions.foreach(extensions.injectFunction)
+ postHocResolutionRuleBuilders.foreach(extensions.injectPostHocResolutionRule)
+ }
+}
diff --git a/gluten-core/src/main/scala/org/apache/spark/util/SparkRuleUtil.scala b/gluten-core/src/main/scala/org/apache/spark/util/SparkPlanRules.scala
similarity index 55%
rename from gluten-core/src/main/scala/org/apache/spark/util/SparkRuleUtil.scala
rename to gluten-core/src/main/scala/org/apache/spark/util/SparkPlanRules.scala
index 100ec36d2424..bbaee81a5987 100644
--- a/gluten-core/src/main/scala/org/apache/spark/util/SparkRuleUtil.scala
+++ b/gluten-core/src/main/scala/org/apache/spark/util/SparkPlanRules.scala
@@ -21,36 +21,48 @@ import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.SparkPlan
-object SparkRuleUtil extends Logging {
-
- /** Add the extended pre/post column rules */
- def extendedColumnarRules(
- session: SparkSession,
- conf: String
- ): List[SparkSession => Rule[SparkPlan]] = {
- val extendedRules = conf.split(",").filter(_.nonEmpty)
- extendedRules
- .map {
- ruleStr =>
+object SparkPlanRules extends Logging {
+ // Since https://github.com/apache/incubator-gluten/pull/1523
+ def extendedColumnarRule(ruleNamesStr: String): SparkSession => Rule[SparkPlan] =
+ (session: SparkSession) => {
+ val ruleNames = ruleNamesStr.split(",").filter(_.nonEmpty)
+ val rules = ruleNames.flatMap {
+ ruleName =>
try {
- val extensionConfClass = Utils.classForName(ruleStr)
- val extensionConf =
- extensionConfClass
+ val ruleClass = Utils.classForName(ruleName)
+ val rule =
+ ruleClass
.getConstructor(classOf[SparkSession])
.newInstance(session)
.asInstanceOf[Rule[SparkPlan]]
-
- Some((sparkSession: SparkSession) => extensionConf)
+ Some(rule)
} catch {
// Ignore the error if we cannot find the class or when the class has the wrong type.
case e @ (_: ClassCastException | _: ClassNotFoundException |
_: NoClassDefFoundError) =>
- logWarning(s"Cannot create extended rule $ruleStr", e)
+ logWarning(s"Cannot create extended rule $ruleName", e)
None
}
}
- .filter(_.isDefined)
- .map(_.get)
- .toList
+ new OrderedRules(rules)
+ }
+
+ object EmptyRule extends Rule[SparkPlan] {
+ override def apply(plan: SparkPlan): SparkPlan = plan
+ }
+
+ class AbortRule(message: String) extends Rule[SparkPlan] {
+ override def apply(plan: SparkPlan): SparkPlan =
+ throw new IllegalStateException(
+ "AbortRule is being executed, this should not happen. Reason: " + message)
+ }
+
+ class OrderedRules(rules: Seq[Rule[SparkPlan]]) extends Rule[SparkPlan] {
+ override def apply(plan: SparkPlan): SparkPlan = {
+ rules.foldLeft(plan) {
+ case (plan, rule) =>
+ rule.apply(plan)
+ }
+ }
}
}
diff --git a/gluten-ut/common/src/test/scala/org/apache/gluten/utils/BackendTestSettings.scala b/gluten-ut/common/src/test/scala/org/apache/gluten/utils/BackendTestSettings.scala
index 987635d067be..dce8ac83710c 100644
--- a/gluten-ut/common/src/test/scala/org/apache/gluten/utils/BackendTestSettings.scala
+++ b/gluten-ut/common/src/test/scala/org/apache/gluten/utils/BackendTestSettings.scala
@@ -30,7 +30,10 @@ abstract class BackendTestSettings {
private val enabledSuites: java.util.Map[String, SuiteSettings] = new util.HashMap()
protected def enableSuite[T: ClassTag]: SuiteSettings = {
- val suiteName = implicitly[ClassTag[T]].runtimeClass.getCanonicalName
+ enableSuite(implicitly[ClassTag[T]].runtimeClass.getCanonicalName)
+ }
+
+ protected def enableSuite(suiteName: String): SuiteSettings = {
if (enabledSuites.containsKey(suiteName)) {
throw new IllegalArgumentException("Duplicated suite name: " + suiteName)
}
diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala
index 7c7aa08791e8..5d171a36bdd4 100644
--- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala
+++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala
@@ -16,8 +16,12 @@
*/
package org.apache.spark.sql.execution
+import org.apache.gluten.GlutenConfig
import org.apache.gluten.execution.BasicScanExecTransformer
import org.apache.gluten.extension.GlutenPlan
+import org.apache.gluten.extension.columnar.{ExpandFallbackPolicy, RemoveFallbackTagRule}
+import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleBuilder
+import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow
import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier
import org.apache.gluten.extension.columnar.transition.InsertTransitions
import org.apache.gluten.utils.QueryPlanSelector
@@ -28,18 +32,20 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
+ import FallbackStrategiesSuite._
testGluten("Fall back the whole query if one unsupported") {
withSQLConf(("spark.gluten.sql.columnar.query.fallback.threshold", "1")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark).withTransformRules(
+ val rule = newRuleApplier(
+ spark,
List(
_ =>
_ => {
UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
},
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ c => InsertTransitions(c.outputsColumnar)))
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to fall back the entire plan.
assert(outputPlan == originalPlan)
}
@@ -48,16 +54,16 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
testGluten("Fall back the whole plan if meeting the configured threshold") {
withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "1")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark)
+ val rule = newRuleApplier(
+ spark,
+ List(
+ _ =>
+ _ => {
+ UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
+ },
+ c => InsertTransitions(c.outputsColumnar)))
.enableAdaptiveContext()
- .withTransformRules(
- List(
- _ =>
- _ => {
- UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
- },
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to fall back the entire plan.
assert(outputPlan == originalPlan)
}
@@ -66,16 +72,16 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
testGluten("Don't fall back the whole plan if NOT meeting the configured threshold") {
withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "4")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark)
+ val rule = newRuleApplier(
+ spark,
+ List(
+ _ =>
+ _ => {
+ UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
+ },
+ c => InsertTransitions(c.outputsColumnar)))
.enableAdaptiveContext()
- .withTransformRules(
- List(
- _ =>
- _ => {
- UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
- },
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to get the plan with columnar rule applied.
assert(outputPlan != originalPlan)
}
@@ -86,16 +92,16 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
" transformable)") {
withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "2")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark)
+ val rule = newRuleApplier(
+ spark,
+ List(
+ _ =>
+ _ => {
+ UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer()))))
+ },
+ c => InsertTransitions(c.outputsColumnar)))
.enableAdaptiveContext()
- .withTransformRules(
- List(
- _ =>
- _ => {
- UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer()))))
- },
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to fall back the entire plan.
assert(outputPlan == originalPlan)
}
@@ -106,16 +112,16 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
"leaf node is transformable)") {
withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "3")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark)
+ val rule = newRuleApplier(
+ spark,
+ List(
+ _ =>
+ _ => {
+ UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer()))))
+ },
+ c => InsertTransitions(c.outputsColumnar)))
.enableAdaptiveContext()
- .withTransformRules(
- List(
- _ =>
- _ => {
- UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer()))))
- },
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to get the plan with columnar rule applied.
assert(outputPlan != originalPlan)
}
@@ -153,43 +159,60 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
}
}
-case class LeafOp(override val supportsColumnar: Boolean = false) extends LeafExecNode {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = Seq.empty
-}
+private object FallbackStrategiesSuite {
+ def newRuleApplier(
+ spark: SparkSession,
+ transformBuilders: Seq[ColumnarRuleBuilder]): HeuristicApplier = {
+ new HeuristicApplier(
+ spark,
+ transformBuilders,
+ List(c => ExpandFallbackPolicy(c.ac.isAdaptiveContext(), c.ac.originalPlan())),
+ List(
+ c => RemoveTopmostColumnarToRow(c.session, c.ac.isAdaptiveContext()),
+ _ => ColumnarCollapseTransformStages(GlutenConfig.getConf)
+ ),
+ List(_ => RemoveFallbackTagRule())
+ )
+ }
-case class UnaryOp1(child: SparkPlan, override val supportsColumnar: Boolean = false)
- extends UnaryExecNode {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = child.output
- override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1 =
- copy(child = newChild)
-}
+ case class LeafOp(override val supportsColumnar: Boolean = false) extends LeafExecNode {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = Seq.empty
+ }
-case class UnaryOp2(child: SparkPlan, override val supportsColumnar: Boolean = false)
- extends UnaryExecNode {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = child.output
- override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp2 =
- copy(child = newChild)
-}
+ case class UnaryOp1(child: SparkPlan, override val supportsColumnar: Boolean = false)
+ extends UnaryExecNode {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = child.output
+ override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1 =
+ copy(child = newChild)
+ }
+
+ case class UnaryOp2(child: SparkPlan, override val supportsColumnar: Boolean = false)
+ extends UnaryExecNode {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = child.output
+ override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp2 =
+ copy(child = newChild)
+ }
// For replacing LeafOp.
-case class LeafOpTransformer(override val supportsColumnar: Boolean = true)
- extends LeafExecNode
- with GlutenPlan {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = Seq.empty
-}
+ case class LeafOpTransformer(override val supportsColumnar: Boolean = true)
+ extends LeafExecNode
+ with GlutenPlan {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = Seq.empty
+ }
// For replacing UnaryOp1.
-case class UnaryOp1Transformer(
- override val child: SparkPlan,
- override val supportsColumnar: Boolean = true)
- extends UnaryExecNode
- with GlutenPlan {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = child.output
- override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer =
- copy(child = newChild)
+ case class UnaryOp1Transformer(
+ override val child: SparkPlan,
+ override val supportsColumnar: Boolean = true)
+ extends UnaryExecNode
+ with GlutenPlan {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = child.output
+ override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer =
+ copy(child = newChild)
+ }
}
diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala
index 6816534094f3..2ca7429f1679 100644
--- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala
+++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala
@@ -31,7 +31,8 @@ class GlutenSessionExtensionSuite extends GlutenSQLTestsTrait {
}
testGluten("test gluten extensions") {
- assert(spark.sessionState.columnarRules.contains(ColumnarOverrideRules(spark)))
+ assert(
+ spark.sessionState.columnarRules.map(_.getClass).contains(classOf[ColumnarOverrideRules]))
assert(spark.sessionState.planner.strategies.contains(MySparkStrategy(spark)))
assert(spark.sessionState.analyzer.extendedResolutionRules.contains(MyRule(spark)))
diff --git a/gluten-ut/spark33/pom.xml b/gluten-ut/spark33/pom.xml
index 9251ebc8ab78..539f60a63f1b 100644
--- a/gluten-ut/spark33/pom.xml
+++ b/gluten-ut/spark33/pom.xml
@@ -51,6 +51,28 @@
test
+
+
+
+ org.codehaus.mojo
+ build-helper-maven-plugin
+
+
+ add-sources
+ generate-sources
+
+ add-test-source
+
+
+
+
+
+
+
+
+
+
+ backends-velox
diff --git a/gluten-ut/spark33/src/test/backends-clickhouse/org/apache/gluten/execution/parquet/GlutenParquetV1FilterSuite2.scala b/gluten-ut/spark33/src/test/backends-clickhouse/org/apache/gluten/execution/parquet/GlutenParquetV1FilterSuite2.scala
new file mode 100644
index 000000000000..32c7784cff92
--- /dev/null
+++ b/gluten-ut/spark33/src/test/backends-clickhouse/org/apache/gluten/execution/parquet/GlutenParquetV1FilterSuite2.scala
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.execution.parquet
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.execution.datasources.parquet.GlutenParquetV1FilterSuite
+
+/** testing use_local_format parquet reader. */
+class GlutenParquetV1FilterSuite2 extends GlutenParquetV1FilterSuite {
+ override def sparkConf: SparkConf =
+ super.sparkConf
+ .set("spark.gluten.sql.columnar.backend.ch.runtime_config.use_local_format", "true")
+}
diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala
index c8e162e61d66..660d693cce3f 100644
--- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala
+++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala
@@ -1600,6 +1600,20 @@ class ClickHouseTestSettings extends BackendTestSettings {
.exclude("SPARK-38825: in and notIn filters")
.exclude("SPARK-36866: filter pushdown - year-month interval")
.excludeGlutenTest("SPARK-25207: exception when duplicate fields in case-insensitive mode")
+ enableSuite("org.apache.gluten.execution.parquet.GlutenParquetV1FilterSuite2")
+ .exclude("filter pushdown - date")
+ .exclude("filter pushdown - timestamp")
+ .exclude("Filters should be pushed down for vectorized Parquet reader at row group level")
+ .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names")
+ .exclude("Filters should be pushed down for Parquet readers at row group level")
+ .exclude("filter pushdown - StringStartsWith")
+ .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down")
+ .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode")
+ .exclude("Support Parquet column index")
+ .exclude("SPARK-34562: Bloom filter push down")
+ .exclude("SPARK-38825: in and notIn filters")
+ .exclude("SPARK-36866: filter pushdown - year-month interval")
+ .excludeGlutenTest("SPARK-25207: exception when duplicate fields in case-insensitive mode")
enableSuite[GlutenParquetV1PartitionDiscoverySuite]
.exclude("SPARK-7847: Dynamic partition directory path escaping and unescaping")
.exclude("Various partition value types")
diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala
index 54d7596b602c..1ce0025f2944 100644
--- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala
+++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala
@@ -16,10 +16,13 @@
*/
package org.apache.spark.sql.execution
+import org.apache.gluten.GlutenConfig
import org.apache.gluten.backendsapi.BackendsApiManager
import org.apache.gluten.execution.BasicScanExecTransformer
import org.apache.gluten.extension.GlutenPlan
-import org.apache.gluten.extension.columnar.{FallbackEmptySchemaRelation, FallbackTags}
+import org.apache.gluten.extension.columnar.{ExpandFallbackPolicy, FallbackEmptySchemaRelation, FallbackTags, RemoveFallbackTagRule}
+import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleBuilder
+import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow
import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier
import org.apache.gluten.extension.columnar.transition.InsertTransitions
import org.apache.gluten.utils.QueryPlanSelector
@@ -30,17 +33,19 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
+ import FallbackStrategiesSuite._
testGluten("Fall back the whole query if one unsupported") {
withSQLConf(("spark.gluten.sql.columnar.query.fallback.threshold", "1")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark).withTransformRules(
+ val rule = newRuleApplier(
+ spark,
List(
_ =>
_ => {
UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
},
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ c => InsertTransitions(c.outputsColumnar)))
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to fall back the entire plan.
assert(outputPlan == originalPlan)
}
@@ -49,16 +54,16 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
testGluten("Fall back the whole plan if meeting the configured threshold") {
withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "1")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark)
+ val rule = newRuleApplier(
+ spark,
+ List(
+ _ =>
+ _ => {
+ UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
+ },
+ c => InsertTransitions(c.outputsColumnar)))
.enableAdaptiveContext()
- .withTransformRules(
- List(
- _ =>
- _ => {
- UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
- },
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to fall back the entire plan.
assert(outputPlan == originalPlan)
}
@@ -67,16 +72,16 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
testGluten("Don't fall back the whole plan if NOT meeting the configured threshold") {
withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "4")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark)
+ val rule = newRuleApplier(
+ spark,
+ List(
+ _ =>
+ _ => {
+ UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
+ },
+ c => InsertTransitions(c.outputsColumnar)))
.enableAdaptiveContext()
- .withTransformRules(
- List(
- _ =>
- _ => {
- UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
- },
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to get the plan with columnar rule applied.
assert(outputPlan != originalPlan)
}
@@ -87,16 +92,16 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
" transformable)") {
withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "2")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark)
+ val rule = newRuleApplier(
+ spark,
+ List(
+ _ =>
+ _ => {
+ UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer()))))
+ },
+ c => InsertTransitions(c.outputsColumnar)))
.enableAdaptiveContext()
- .withTransformRules(
- List(
- _ =>
- _ => {
- UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer()))))
- },
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to fall back the entire plan.
assert(outputPlan == originalPlan)
}
@@ -107,16 +112,16 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
"leaf node is transformable)") {
withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "3")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark)
+ val rule = newRuleApplier(
+ spark,
+ List(
+ _ =>
+ _ => {
+ UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer()))))
+ },
+ c => InsertTransitions(c.outputsColumnar)))
.enableAdaptiveContext()
- .withTransformRules(
- List(
- _ =>
- _ => {
- UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer()))))
- },
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to get the plan with columnar rule applied.
assert(outputPlan != originalPlan)
}
@@ -168,44 +173,60 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
thread.join(10000)
}
}
+private object FallbackStrategiesSuite {
+ def newRuleApplier(
+ spark: SparkSession,
+ transformBuilders: Seq[ColumnarRuleBuilder]): HeuristicApplier = {
+ new HeuristicApplier(
+ spark,
+ transformBuilders,
+ List(c => ExpandFallbackPolicy(c.ac.isAdaptiveContext(), c.ac.originalPlan())),
+ List(
+ c => RemoveTopmostColumnarToRow(c.session, c.ac.isAdaptiveContext()),
+ _ => ColumnarCollapseTransformStages(GlutenConfig.getConf)
+ ),
+ List(_ => RemoveFallbackTagRule())
+ )
+ }
-case class LeafOp(override val supportsColumnar: Boolean = false) extends LeafExecNode {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = Seq.empty
-}
+ case class LeafOp(override val supportsColumnar: Boolean = false) extends LeafExecNode {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = Seq.empty
+ }
-case class UnaryOp1(child: SparkPlan, override val supportsColumnar: Boolean = false)
- extends UnaryExecNode {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = child.output
- override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1 =
- copy(child = newChild)
-}
+ case class UnaryOp1(child: SparkPlan, override val supportsColumnar: Boolean = false)
+ extends UnaryExecNode {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = child.output
+ override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1 =
+ copy(child = newChild)
+ }
-case class UnaryOp2(child: SparkPlan, override val supportsColumnar: Boolean = false)
- extends UnaryExecNode {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = child.output
- override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp2 =
- copy(child = newChild)
-}
+ case class UnaryOp2(child: SparkPlan, override val supportsColumnar: Boolean = false)
+ extends UnaryExecNode {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = child.output
+ override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp2 =
+ copy(child = newChild)
+ }
-// For replacing LeafOp.
-case class LeafOpTransformer(override val supportsColumnar: Boolean = true)
- extends LeafExecNode
- with GlutenPlan {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = Seq.empty
-}
+ // For replacing LeafOp.
+ case class LeafOpTransformer(override val supportsColumnar: Boolean = true)
+ extends LeafExecNode
+ with GlutenPlan {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = Seq.empty
+ }
-// For replacing UnaryOp1.
-case class UnaryOp1Transformer(
- override val child: SparkPlan,
- override val supportsColumnar: Boolean = true)
- extends UnaryExecNode
- with GlutenPlan {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = child.output
- override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer =
- copy(child = newChild)
+ // For replacing UnaryOp1.
+ case class UnaryOp1Transformer(
+ override val child: SparkPlan,
+ override val supportsColumnar: Boolean = true)
+ extends UnaryExecNode
+ with GlutenPlan {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = child.output
+ override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer =
+ copy(child = newChild)
+ }
}
diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala
index 6816534094f3..2ca7429f1679 100644
--- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala
+++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala
@@ -31,7 +31,8 @@ class GlutenSessionExtensionSuite extends GlutenSQLTestsTrait {
}
testGluten("test gluten extensions") {
- assert(spark.sessionState.columnarRules.contains(ColumnarOverrideRules(spark)))
+ assert(
+ spark.sessionState.columnarRules.map(_.getClass).contains(classOf[ColumnarOverrideRules]))
assert(spark.sessionState.planner.strategies.contains(MySparkStrategy(spark)))
assert(spark.sessionState.analyzer.extendedResolutionRules.contains(MyRule(spark)))
diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala
index 5150a4768851..3acc9c4b39aa 100644
--- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala
+++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala
@@ -16,10 +16,13 @@
*/
package org.apache.spark.sql.execution
+import org.apache.gluten.GlutenConfig
import org.apache.gluten.backendsapi.BackendsApiManager
import org.apache.gluten.execution.BasicScanExecTransformer
import org.apache.gluten.extension.GlutenPlan
-import org.apache.gluten.extension.columnar.{FallbackEmptySchemaRelation, FallbackTags}
+import org.apache.gluten.extension.columnar.{ExpandFallbackPolicy, FallbackEmptySchemaRelation, FallbackTags, RemoveFallbackTagRule}
+import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleBuilder
+import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow
import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier
import org.apache.gluten.extension.columnar.transition.InsertTransitions
import org.apache.gluten.utils.QueryPlanSelector
@@ -30,18 +33,19 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
-
+ import FallbackStrategiesSuite._
testGluten("Fall back the whole query if one unsupported") {
withSQLConf(("spark.gluten.sql.columnar.query.fallback.threshold", "1")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark).withTransformRules(
+ val rule = newRuleApplier(
+ spark,
List(
_ =>
_ => {
UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
},
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ c => InsertTransitions(c.outputsColumnar)))
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to fall back the entire plan.
assert(outputPlan == originalPlan)
}
@@ -50,16 +54,16 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
testGluten("Fall back the whole plan if meeting the configured threshold") {
withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "1")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark)
+ val rule = newRuleApplier(
+ spark,
+ List(
+ _ =>
+ _ => {
+ UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
+ },
+ c => InsertTransitions(c.outputsColumnar)))
.enableAdaptiveContext()
- .withTransformRules(
- List(
- _ =>
- _ => {
- UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
- },
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to fall back the entire plan.
assert(outputPlan == originalPlan)
}
@@ -68,16 +72,16 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
testGluten("Don't fall back the whole plan if NOT meeting the configured threshold") {
withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "4")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark)
+ val rule = newRuleApplier(
+ spark,
+ List(
+ _ =>
+ _ => {
+ UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
+ },
+ c => InsertTransitions(c.outputsColumnar)))
.enableAdaptiveContext()
- .withTransformRules(
- List(
- _ =>
- _ => {
- UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
- },
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to get the plan with columnar rule applied.
assert(outputPlan != originalPlan)
}
@@ -88,16 +92,16 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
" transformable)") {
withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "2")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark)
+ val rule = newRuleApplier(
+ spark,
+ List(
+ _ =>
+ _ => {
+ UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer()))))
+ },
+ c => InsertTransitions(c.outputsColumnar)))
.enableAdaptiveContext()
- .withTransformRules(
- List(
- _ =>
- _ => {
- UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer()))))
- },
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to fall back the entire plan.
assert(outputPlan == originalPlan)
}
@@ -108,16 +112,16 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
"leaf node is transformable)") {
withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "3")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark)
+ val rule = newRuleApplier(
+ spark,
+ List(
+ _ =>
+ _ => {
+ UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer()))))
+ },
+ c => InsertTransitions(c.outputsColumnar)))
.enableAdaptiveContext()
- .withTransformRules(
- List(
- _ =>
- _ => {
- UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer()))))
- },
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to get the plan with columnar rule applied.
assert(outputPlan != originalPlan)
}
@@ -170,43 +174,60 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
}
}
-case class LeafOp(override val supportsColumnar: Boolean = false) extends LeafExecNode {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = Seq.empty
-}
+private object FallbackStrategiesSuite {
+ def newRuleApplier(
+ spark: SparkSession,
+ transformBuilders: Seq[ColumnarRuleBuilder]): HeuristicApplier = {
+ new HeuristicApplier(
+ spark,
+ transformBuilders,
+ List(c => ExpandFallbackPolicy(c.ac.isAdaptiveContext(), c.ac.originalPlan())),
+ List(
+ c => RemoveTopmostColumnarToRow(c.session, c.ac.isAdaptiveContext()),
+ _ => ColumnarCollapseTransformStages(GlutenConfig.getConf)
+ ),
+ List(_ => RemoveFallbackTagRule())
+ )
+ }
-case class UnaryOp1(child: SparkPlan, override val supportsColumnar: Boolean = false)
- extends UnaryExecNode {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = child.output
- override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1 =
- copy(child = newChild)
-}
+ case class LeafOp(override val supportsColumnar: Boolean = false) extends LeafExecNode {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = Seq.empty
+ }
-case class UnaryOp2(child: SparkPlan, override val supportsColumnar: Boolean = false)
- extends UnaryExecNode {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = child.output
- override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp2 =
- copy(child = newChild)
-}
+ case class UnaryOp1(child: SparkPlan, override val supportsColumnar: Boolean = false)
+ extends UnaryExecNode {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = child.output
+ override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1 =
+ copy(child = newChild)
+ }
-// For replacing LeafOp.
-case class LeafOpTransformer(override val supportsColumnar: Boolean = true)
- extends LeafExecNode
- with GlutenPlan {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = Seq.empty
-}
+ case class UnaryOp2(child: SparkPlan, override val supportsColumnar: Boolean = false)
+ extends UnaryExecNode {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = child.output
+ override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp2 =
+ copy(child = newChild)
+ }
-// For replacing UnaryOp1.
-case class UnaryOp1Transformer(
- override val child: SparkPlan,
- override val supportsColumnar: Boolean = true)
- extends UnaryExecNode
- with GlutenPlan {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = child.output
- override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer =
- copy(child = newChild)
+ // For replacing LeafOp.
+ case class LeafOpTransformer(override val supportsColumnar: Boolean = true)
+ extends LeafExecNode
+ with GlutenPlan {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = Seq.empty
+ }
+
+ // For replacing UnaryOp1.
+ case class UnaryOp1Transformer(
+ override val child: SparkPlan,
+ override val supportsColumnar: Boolean = true)
+ extends UnaryExecNode
+ with GlutenPlan {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = child.output
+ override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer =
+ copy(child = newChild)
+ }
}
diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala
index 6816534094f3..2ca7429f1679 100644
--- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala
+++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala
@@ -31,7 +31,8 @@ class GlutenSessionExtensionSuite extends GlutenSQLTestsTrait {
}
testGluten("test gluten extensions") {
- assert(spark.sessionState.columnarRules.contains(ColumnarOverrideRules(spark)))
+ assert(
+ spark.sessionState.columnarRules.map(_.getClass).contains(classOf[ColumnarOverrideRules]))
assert(spark.sessionState.planner.strategies.contains(MySparkStrategy(spark)))
assert(spark.sessionState.analyzer.extendedResolutionRules.contains(MyRule(spark)))
diff --git a/gluten-ut/spark35/src/test/backends-clickhouse/org/apache/gluten/GlutenColumnarWriteTestSupport.scala b/gluten-ut/spark35/src/test/backends-clickhouse/org/apache/gluten/GlutenColumnarWriteTestSupport.scala
index 43b83afe9af3..4258cd891a5a 100644
--- a/gluten-ut/spark35/src/test/backends-clickhouse/org/apache/gluten/GlutenColumnarWriteTestSupport.scala
+++ b/gluten-ut/spark35/src/test/backends-clickhouse/org/apache/gluten/GlutenColumnarWriteTestSupport.scala
@@ -16,11 +16,12 @@
*/
package org.apache.gluten
-import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.{ColumnarWriteFilesExec, SparkPlan}
trait GlutenColumnarWriteTestSupport {
def checkWriteFilesAndGetChild(sparkPlan: SparkPlan): SparkPlan = {
- throw new UnsupportedOperationException("Clickhouse Backend does not support write files")
+ assert(sparkPlan.isInstanceOf[ColumnarWriteFilesExec])
+ sparkPlan.asInstanceOf[ColumnarWriteFilesExec].child
}
}
diff --git a/gluten-ut/spark35/src/test/backends-clickhouse/org/apache/gluten/execution/parquet/GlutenParquetV1FilterSuite2.scala b/gluten-ut/spark35/src/test/backends-clickhouse/org/apache/gluten/execution/parquet/GlutenParquetV1FilterSuite2.scala
new file mode 100644
index 000000000000..d20a419597d1
--- /dev/null
+++ b/gluten-ut/spark35/src/test/backends-clickhouse/org/apache/gluten/execution/parquet/GlutenParquetV1FilterSuite2.scala
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.execution.parquet
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.execution.datasources.parquet.GlutenParquetV1FilterSuite
+
+
+/** testing use_local_format parquet reader.
+ * FIXME: Run this suite in Spark 35 CI Pipeline
+ * */
+class GlutenParquetV1FilterSuite2 extends GlutenParquetV1FilterSuite {
+ override def sparkConf: SparkConf =
+ super.sparkConf
+ .set("spark.gluten.sql.columnar.backend.ch.runtime_config.use_local_format", "true")
+}
diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala
index 77c12621efeb..bf971aba7282 100644
--- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala
+++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala
@@ -1438,6 +1438,8 @@ class ClickHouseTestSettings extends BackendTestSettings {
.exclude("SPARK-34562: Bloom filter push down")
.exclude("SPARK-38825: in and notIn filters")
.exclude("SPARK-36866: filter pushdown - year-month interval")
+ .exclude("filter pushdown - StringContains")
+ .exclude("filter pushdown - StringPredicate")
.excludeGlutenTest("SPARK-25207: exception when duplicate fields in case-insensitive mode")
enableSuite[GlutenParquetV1PartitionDiscoverySuite]
.exclude("SPARK-7847: Dynamic partition directory path escaping and unescaping")
diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala
index 5150a4768851..bcc4e829b535 100644
--- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala
+++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala
@@ -16,10 +16,13 @@
*/
package org.apache.spark.sql.execution
+import org.apache.gluten.GlutenConfig
import org.apache.gluten.backendsapi.BackendsApiManager
import org.apache.gluten.execution.BasicScanExecTransformer
import org.apache.gluten.extension.GlutenPlan
-import org.apache.gluten.extension.columnar.{FallbackEmptySchemaRelation, FallbackTags}
+import org.apache.gluten.extension.columnar.{ExpandFallbackPolicy, FallbackEmptySchemaRelation, FallbackTags, RemoveFallbackTagRule}
+import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleBuilder
+import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow
import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier
import org.apache.gluten.extension.columnar.transition.InsertTransitions
import org.apache.gluten.utils.QueryPlanSelector
@@ -30,18 +33,20 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
+ import FallbackStrategiesSuite._
testGluten("Fall back the whole query if one unsupported") {
withSQLConf(("spark.gluten.sql.columnar.query.fallback.threshold", "1")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark).withTransformRules(
+ val rule = newRuleApplier(
+ spark,
List(
_ =>
_ => {
UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
},
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ c => InsertTransitions(c.outputsColumnar)))
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to fall back the entire plan.
assert(outputPlan == originalPlan)
}
@@ -50,16 +55,16 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
testGluten("Fall back the whole plan if meeting the configured threshold") {
withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "1")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark)
+ val rule = newRuleApplier(
+ spark,
+ List(
+ _ =>
+ _ => {
+ UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
+ },
+ c => InsertTransitions(c.outputsColumnar)))
.enableAdaptiveContext()
- .withTransformRules(
- List(
- _ =>
- _ => {
- UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
- },
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to fall back the entire plan.
assert(outputPlan == originalPlan)
}
@@ -68,16 +73,16 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
testGluten("Don't fall back the whole plan if NOT meeting the configured threshold") {
withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "4")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark)
+ val rule = newRuleApplier(
+ spark,
+ List(
+ _ =>
+ _ => {
+ UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
+ },
+ c => InsertTransitions(c.outputsColumnar)))
.enableAdaptiveContext()
- .withTransformRules(
- List(
- _ =>
- _ => {
- UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp()))))
- },
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to get the plan with columnar rule applied.
assert(outputPlan != originalPlan)
}
@@ -88,16 +93,16 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
" transformable)") {
withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "2")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark)
+ val rule = newRuleApplier(
+ spark,
+ List(
+ _ =>
+ _ => {
+ UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer()))))
+ },
+ c => InsertTransitions(c.outputsColumnar)))
.enableAdaptiveContext()
- .withTransformRules(
- List(
- _ =>
- _ => {
- UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer()))))
- },
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to fall back the entire plan.
assert(outputPlan == originalPlan)
}
@@ -108,16 +113,16 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
"leaf node is transformable)") {
withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "3")) {
val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp()))))
- val rule = new HeuristicApplier(spark)
+ val rule = newRuleApplier(
+ spark,
+ List(
+ _ =>
+ _ => {
+ UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer()))))
+ },
+ c => InsertTransitions(c.outputsColumnar)))
.enableAdaptiveContext()
- .withTransformRules(
- List(
- _ =>
- _ => {
- UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer()))))
- },
- (_: SparkSession) => InsertTransitions(outputsColumnar = false)))
- val outputPlan = rule.apply(originalPlan)
+ val outputPlan = rule.apply(originalPlan, false)
// Expect to get the plan with columnar rule applied.
assert(outputPlan != originalPlan)
}
@@ -170,43 +175,60 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait {
}
}
-case class LeafOp(override val supportsColumnar: Boolean = false) extends LeafExecNode {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = Seq.empty
-}
+private object FallbackStrategiesSuite {
+ def newRuleApplier(
+ spark: SparkSession,
+ transformBuilders: Seq[ColumnarRuleBuilder]): HeuristicApplier = {
+ new HeuristicApplier(
+ spark,
+ transformBuilders,
+ List(c => ExpandFallbackPolicy(c.ac.isAdaptiveContext(), c.ac.originalPlan())),
+ List(
+ c => RemoveTopmostColumnarToRow(c.session, c.ac.isAdaptiveContext()),
+ _ => ColumnarCollapseTransformStages(GlutenConfig.getConf)
+ ),
+ List(_ => RemoveFallbackTagRule())
+ )
+ }
-case class UnaryOp1(child: SparkPlan, override val supportsColumnar: Boolean = false)
- extends UnaryExecNode {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = child.output
- override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1 =
- copy(child = newChild)
-}
+ case class LeafOp(override val supportsColumnar: Boolean = false) extends LeafExecNode {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = Seq.empty
+ }
-case class UnaryOp2(child: SparkPlan, override val supportsColumnar: Boolean = false)
- extends UnaryExecNode {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = child.output
- override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp2 =
- copy(child = newChild)
-}
+ case class UnaryOp1(child: SparkPlan, override val supportsColumnar: Boolean = false)
+ extends UnaryExecNode {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = child.output
+ override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1 =
+ copy(child = newChild)
+ }
+
+ case class UnaryOp2(child: SparkPlan, override val supportsColumnar: Boolean = false)
+ extends UnaryExecNode {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = child.output
+ override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp2 =
+ copy(child = newChild)
+ }
// For replacing LeafOp.
-case class LeafOpTransformer(override val supportsColumnar: Boolean = true)
- extends LeafExecNode
- with GlutenPlan {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = Seq.empty
-}
+ case class LeafOpTransformer(override val supportsColumnar: Boolean = true)
+ extends LeafExecNode
+ with GlutenPlan {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = Seq.empty
+ }
// For replacing UnaryOp1.
-case class UnaryOp1Transformer(
- override val child: SparkPlan,
- override val supportsColumnar: Boolean = true)
- extends UnaryExecNode
- with GlutenPlan {
- override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
- override def output: Seq[Attribute] = child.output
- override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer =
- copy(child = newChild)
+ case class UnaryOp1Transformer(
+ override val child: SparkPlan,
+ override val supportsColumnar: Boolean = true)
+ extends UnaryExecNode
+ with GlutenPlan {
+ override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
+ override def output: Seq[Attribute] = child.output
+ override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer =
+ copy(child = newChild)
+ }
}
diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala
index 6816534094f3..2ca7429f1679 100644
--- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala
+++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala
@@ -31,7 +31,8 @@ class GlutenSessionExtensionSuite extends GlutenSQLTestsTrait {
}
testGluten("test gluten extensions") {
- assert(spark.sessionState.columnarRules.contains(ColumnarOverrideRules(spark)))
+ assert(
+ spark.sessionState.columnarRules.map(_.getClass).contains(classOf[ColumnarOverrideRules]))
assert(spark.sessionState.planner.strategies.contains(MySparkStrategy(spark)))
assert(spark.sessionState.analyzer.extendedResolutionRules.contains(MyRule(spark)))
diff --git a/package/pom.xml b/package/pom.xml
index 0436889c090b..f385a2a5a058 100644
--- a/package/pom.xml
+++ b/package/pom.xml
@@ -186,7 +186,12 @@
META-INF/*.SFMETA-INF/*.DSA
- META-INF/*.RSA
+ META-INF/*.RSA
+ META-INF/DEPENDENCIES
+ META-INF/LICENSE.txt
+ META-INF/NOTICE.txt
+ LICENSE.txt
+ NOTICE.txt
diff --git a/package/src/main/resources/META-INF/LICENSE b/package/src/main/resources/META-INF/LICENSE
new file mode 100644
index 000000000000..3680275b939a
--- /dev/null
+++ b/package/src/main/resources/META-INF/LICENSE
@@ -0,0 +1,262 @@
+
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ Licensed 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.
+
+---------------------------------------------------------
+
+This project bundles some components that are also licensed under the Apache
+License Version 2.0:
+
+com.fasterxml.jackson.core:jackson-core
+com.fasterxml.jackson.core:jackson-databind
+com.fasterxml.jackson.datatype:jackson-datatype-jsr310
+com.fasterxml.jackson.module:jackson-module-scala_2.12
+com.google.code.findbugs:jsr305
+com.google.code.gson:gson
+com.google.errorprone:error_prone_annotations
+com.google.flatbuffers:flatbuffers-java
+com.google.guava:failureaccess
+com.google.guava:guava
+com.google.guava:listenablefuture
+com.google.j2objc:j2objc-annotations
+com.google.jimfs:jimfs
+com.github.ben-manes.caffeine:caffeine
+commons-codec:commons-codec
+info.picocli:picocli
+io.trino.tpcds:tpcds
+io.trino.tpch:tpch
+javax.inject:javax.inject
+org.scala-lang:scala-library
+org.apache.arrow:arrow-format
+org.apache.arrow:arrow-memory-core
+org.apache.arrow:arrow-memory-unsafe
+org.apache.arrow:arrow-vector
+
+---------------------------------------------------------
+
+This product bundles various third-party components under other open source licenses.
+This section summarizes those components and their licenses. See licenses-binary/
+for text of these licenses.
+
+BSD 3-Clause
+------------
+
+com.thoughtworks.paranamer:paranamer
+io.glutenproject:protobuf-java
+io.glutenproject:protobuf-java-util
+org.eclipse.collections:eclipse-collections
+org.eclipse.collections:eclipse-collections-api
+
+
+MIT License
+-----------
+
+org.checkerframework:checker-qual
+org.slf4j:slf4j-api
+
+
+Eclipse Public License (EPL) 1.0
+--------------------------------
+
+org.eclipse.collections:eclipse-collections
+org.eclipse.collections:eclipse-collections-api
+
diff --git a/package/src/main/resources/META-INF/NOTICE b/package/src/main/resources/META-INF/NOTICE
new file mode 100644
index 000000000000..be510d300f22
--- /dev/null
+++ b/package/src/main/resources/META-INF/NOTICE
@@ -0,0 +1,1722 @@
+Apache Gluten(incubating)
+Copyright 2023-2024 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+## Third-party Content
+
+This project leverages the following third party content.
+
+Apache Spark
+Copyright 2014 and onwards The Apache Software Foundation.
+
+---------------------------------------------------------
+
+Apache Celeborn
+Copyright 2022-2024 The Apache Software Foundation.
+
+---------------------------------------------------------
+
+Apache Uniffle (incubating)
+Copyright 2022 and onwards The Apache Software Foundation.
+
+---------------------------------------------------------
+
+Apache Iceberg
+Copyright 2017-2024 The Apache Software Foundation.
+
+---------------------------------------------------------
+
+Apache Parquet MR
+Copyright 2014-2024 The Apache Software Foundation.
+
+---------------------------------------------------------
+
+Apache ORC
+Copyright 2013 and onwards The Apache Software Foundation.
+
+---------------------------------------------------------
+
+Apache Thrift
+Copyright (C) 2006 - 2019, The Apache Software Foundation.
+
+---------------------------------------------------------
+
+This project includes code from Daniel Lemire's FrameOfReference project.
+
+https://github.com/lemire/FrameOfReference/blob/6ccaf9e97160f9a3b299e23a8ef739e711ef0c71/src/bpacking.cpp
+
+Copyright: 2013 Daniel Lemire
+Home page: http://lemire.me/en/
+Project page: https://github.com/lemire/FrameOfReference
+License: Apache License Version 2.0 http://www.apache.org/licenses/LICENSE-2.0
+
+---------------------------------------------------------
+
+This project includes code from the TensorFlow project
+
+Copyright 2015 The TensorFlow Authors. All Rights Reserved.
+
+Licensed 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.
+
+---------------------------------------------------------
+
+This project includes code from the NumPy project.
+
+https://github.com/numpy/numpy/blob/e1f191c46f2eebd6cb892a4bfe14d9dd43a06c4e/numpy/core/src/multiarray/multiarraymodule.c#L2910
+
+https://github.com/numpy/numpy/blob/68fd82271b9ea5a9e50d4e761061dfcca851382a/numpy/core/src/multiarray/datetime.c
+
+Copyright (c) 2005-2017, NumPy Developers.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+ * Redistributions of source code must retain the above copyright
+ notice, this list of conditions and the following disclaimer.
+
+ * Redistributions in binary form must reproduce the above
+ copyright notice, this list of conditions and the following
+ disclaimer in the documentation and/or other materials provided
+ with the distribution.
+
+ * Neither the name of the NumPy Developers nor the names of any
+ contributors may be used to endorse or promote products derived
+ from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+---------------------------------------------------------
+
+This project includes code from the Boost project
+
+Boost Software License - Version 1.0 - August 17th, 2003
+
+Permission is hereby granted, free of charge, to any person or organization
+obtaining a copy of the software and accompanying documentation covered by
+this license (the "Software") to use, reproduce, display, distribute,
+execute, and transmit the Software, and to prepare derivative works of the
+Software, and to permit third-parties to whom the Software is furnished to
+do so, all subject to the following:
+
+The copyright notices in the Software and this entire statement, including
+the above license grant, this restriction and the following disclaimer,
+must be included in all copies of the Software, in whole or in part, and
+all derivative works of the Software, unless such copies or derivative
+works are solely in the form of machine-executable object code generated by
+a source language processor.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT
+SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE
+FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE,
+ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
+DEALINGS IN THE SOFTWARE.
+
+---------------------------------------------------------
+
+This project includes code from the FlatBuffers project
+
+Copyright 2014 Google Inc.
+
+Licensed 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.
+
+---------------------------------------------------------
+
+This project includes code from the tslib project
+
+Copyright 2015 Microsoft Corporation. All rights reserved.
+
+Licensed 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.
+
+---------------------------------------------------------
+
+This project includes code from the jemalloc project
+
+https://github.com/jemalloc/jemalloc
+
+Copyright (C) 2002-2017 Jason Evans .
+All rights reserved.
+Copyright (C) 2007-2012 Mozilla Foundation. All rights reserved.
+Copyright (C) 2009-2017 Facebook, Inc. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+1. Redistributions of source code must retain the above copyright notice(s),
+ this list of conditions and the following disclaimer.
+2. Redistributions in binary form must reproduce the above copyright notice(s),
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDER(S) ``AS IS'' AND ANY EXPRESS
+OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
+MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO
+EVENT SHALL THE COPYRIGHT HOLDER(S) BE LIABLE FOR ANY DIRECT, INDIRECT,
+INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE
+OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF
+ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+---------------------------------------------------------
+
+This project includes code from the Go project, BSD 3-clause license + PATENTS
+weak patent termination clause
+(https://github.com/golang/go/blob/master/PATENTS).
+
+Copyright (c) 2009 The Go Authors. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+ * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+ * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+ * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+---------------------------------------------------------
+
+This project includes code from the hs2client
+
+https://github.com/cloudera/hs2client
+
+Copyright 2016 Cloudera Inc.
+
+Licensed 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.
+
+--------------------------------------------------------------------------------
+
+The script ci/scripts/util_wait_for_it.sh has the following license
+
+Copyright (c) 2016 Giles Hall
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
+of the Software, and to permit persons to whom the Software is furnished to do
+so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+The script r/configure has the following license (MIT)
+
+Copyright (c) 2017, Jeroen Ooms and Jim Hester
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
+of the Software, and to permit persons to whom the Software is furnished to do
+so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+cpp/src/arrow/util/logging.cc, cpp/src/arrow/util/logging.h and
+cpp/src/arrow/util/logging-test.cc are adapted from
+Ray Project (https://github.com/ray-project/ray) (Apache 2.0).
+
+Copyright (c) 2016 Ray Project (https://github.com/ray-project/ray)
+
+Licensed 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.
+
+--------------------------------------------------------------------------------
+The files cpp/src/arrow/vendored/datetime/date.h, cpp/src/arrow/vendored/datetime/tz.h,
+cpp/src/arrow/vendored/datetime/tz_private.h, cpp/src/arrow/vendored/datetime/ios.h,
+cpp/src/arrow/vendored/datetime/tz.cpp are adapted from
+Howard Hinnant's date library (https://github.com/HowardHinnant/date)
+It is licensed under MIT license.
+
+The MIT License (MIT)
+Copyright (c) 2015, 2016, 2017 Howard Hinnant
+Copyright (c) 2016 Adrian Colomitchi
+Copyright (c) 2017 Florian Dang
+Copyright (c) 2017 Paul Thompson
+Copyright (c) 2018 Tomasz Kamiński
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+The file cpp/src/arrow/util/utf8.h includes code adapted from the page
+ https://bjoern.hoehrmann.de/utf-8/decoder/dfa/
+with the following license (MIT)
+
+Copyright (c) 2008-2009 Bjoern Hoehrmann
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+The file cpp/src/arrow/vendored/string_view.hpp has the following license
+
+Boost Software License - Version 1.0 - August 17th, 2003
+
+Permission is hereby granted, free of charge, to any person or organization
+obtaining a copy of the software and accompanying documentation covered by
+this license (the "Software") to use, reproduce, display, distribute,
+execute, and transmit the Software, and to prepare derivative works of the
+Software, and to permit third-parties to whom the Software is furnished to
+do so, all subject to the following:
+
+The copyright notices in the Software and this entire statement, including
+the above license grant, this restriction and the following disclaimer,
+must be included in all copies of the Software, in whole or in part, and
+all derivative works of the Software, unless such copies or derivative
+works are solely in the form of machine-executable object code generated by
+a source language processor.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT
+SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE
+FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE,
+ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
+DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+The file cpp/src/arrow/vendored/variant.hpp has the following license
+
+Boost Software License - Version 1.0 - August 17th, 2003
+
+Permission is hereby granted, free of charge, to any person or organization
+obtaining a copy of the software and accompanying documentation covered by
+this license (the "Software") to use, reproduce, display, distribute,
+execute, and transmit the Software, and to prepare derivative works of the
+Software, and to permit third-parties to whom the Software is furnished to
+do so, all subject to the following:
+
+The copyright notices in the Software and this entire statement, including
+the above license grant, this restriction and the following disclaimer,
+must be included in all copies of the Software, in whole or in part, and
+all derivative works of the Software, unless such copies or derivative
+works are solely in the form of machine-executable object code generated by
+a source language processor.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT
+SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE
+FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE,
+ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
+DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+The files in cpp/src/arrow/vendored/xxhash/ have the following license
+(BSD 2-Clause License)
+
+xxHash Library
+Copyright (c) 2012-2014, Yann Collet
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice, this
+ list of conditions and the following disclaimer in the documentation and/or
+ other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+You can contact the author at :
+- xxHash homepage: http://www.xxhash.com
+- xxHash source repository : https://github.com/Cyan4973/xxHash
+
+--------------------------------------------------------------------------------
+
+The files in cpp/src/arrow/vendored/double-conversion/ have the following license
+(BSD 3-Clause License)
+
+Copyright 2006-2011, the V8 project authors. All rights reserved.
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+ * Redistributions of source code must retain the above copyright
+ notice, this list of conditions and the following disclaimer.
+ * Redistributions in binary form must reproduce the above
+ copyright notice, this list of conditions and the following
+ disclaimer in the documentation and/or other materials provided
+ with the distribution.
+ * Neither the name of Google Inc. nor the names of its
+ contributors may be used to endorse or promote products derived
+ from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+The files in cpp/src/arrow/vendored/uriparser/ have the following license
+(BSD 3-Clause License)
+
+uriparser - RFC 3986 URI parsing library
+
+Copyright (C) 2007, Weijia Song
+Copyright (C) 2007, Sebastian Pipping
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions
+are met:
+
+ * Redistributions of source code must retain the above
+ copyright notice, this list of conditions and the following
+ disclaimer.
+
+ * Redistributions in binary form must reproduce the above
+ copyright notice, this list of conditions and the following
+ disclaimer in the documentation and/or other materials
+ provided with the distribution.
+
+ * Neither the name of the nor the names of its
+ contributors may be used to endorse or promote products
+ derived from this software without specific prior written
+ permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS
+FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE
+COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT,
+INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
+HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
+OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+The files under dev/tasks/conda-recipes have the following license
+
+BSD 3-clause license
+Copyright (c) 2015-2018, conda-forge
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+1. Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+2. Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+3. Neither the name of the copyright holder nor the names of its contributors
+ may be used to endorse or promote products derived from this software without
+ specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR
+TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+The files in cpp/src/arrow/vendored/utf8cpp/ have the following license
+
+Copyright 2006 Nemanja Trifunovic
+
+Permission is hereby granted, free of charge, to any person or organization
+obtaining a copy of the software and accompanying documentation covered by
+this license (the "Software") to use, reproduce, display, distribute,
+execute, and transmit the Software, and to prepare derivative works of the
+Software, and to permit third-parties to whom the Software is furnished to
+do so, all subject to the following:
+
+The copyright notices in the Software and this entire statement, including
+the above license grant, this restriction and the following disclaimer,
+must be included in all copies of the Software, in whole or in part, and
+all derivative works of the Software, unless such copies or derivative
+works are solely in the form of machine-executable object code generated by
+a source language processor.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT
+SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE
+FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE,
+ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
+DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+This project includes code from Apache Kudu.
+
+ * cpp/cmake_modules/CompilerInfo.cmake is based on Kudu's cmake_modules/CompilerInfo.cmake
+
+Copyright: 2016 The Apache Software Foundation.
+Home page: https://kudu.apache.org/
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This project includes code from Apache Impala (incubating), formerly
+Impala. The Impala code and rights were donated to the ASF as part of the
+Incubator process after the initial code imports into Apache Parquet.
+
+Copyright: 2012 Cloudera, Inc.
+Copyright: 2016 The Apache Software Foundation.
+Home page: http://impala.apache.org/
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This project includes code from Apache Aurora.
+
+* dev/release/{release,changelog,release-candidate} are based on the scripts from
+ Apache Aurora
+
+Copyright: 2016 The Apache Software Foundation.
+Home page: https://aurora.apache.org/
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This project includes code from the Google styleguide.
+
+* cpp/build-support/cpplint.py is based on the scripts from the Google styleguide.
+
+Copyright: 2009 Google Inc. All rights reserved.
+Homepage: https://github.com/google/styleguide
+License: 3-clause BSD
+
+--------------------------------------------------------------------------------
+
+This project includes code from Snappy.
+
+* cpp/cmake_modules/{SnappyCMakeLists.txt,SnappyConfig.h} are based on code
+ from Google's Snappy project.
+
+Copyright: 2009 Google Inc. All rights reserved.
+Homepage: https://github.com/google/snappy
+License: 3-clause BSD
+
+--------------------------------------------------------------------------------
+
+This project includes code from the manylinux project.
+
+* python/manylinux1/scripts/{build_python.sh,python-tag-abi-tag.py,
+ requirements.txt} are based on code from the manylinux project.
+
+Copyright: 2016 manylinux
+Homepage: https://github.com/pypa/manylinux
+License: The MIT License (MIT)
+
+--------------------------------------------------------------------------------
+
+This project includes code from the cymove project:
+
+* python/pyarrow/includes/common.pxd includes code from the cymove project
+
+The MIT License (MIT)
+Copyright (c) 2019 Omer Ozarslan
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM,
+DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
+OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE
+OR OTHER DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+The projects includes code from the Ursabot project under the dev/archery
+directory.
+
+License: BSD 2-Clause
+
+Copyright 2019 RStudio, Inc.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+1. Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+2. Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+This project include code from CMake.
+
+* cpp/cmake_modules/FindGTest.cmake is based on code from CMake.
+
+Copyright: Copyright 2000-2019 Kitware, Inc. and Contributors
+Homepage: https://gitlab.kitware.com/cmake/cmake
+License: 3-clause BSD
+
+--------------------------------------------------------------------------------
+
+This project include code from mingw-w64.
+
+* cpp/src/arrow/util/cpu-info.cc has a polyfill for mingw-w64 < 5
+
+Copyright (c) 2009 - 2013 by the mingw-w64 project
+Homepage: https://mingw-w64.org
+License: Zope Public License (ZPL) Version 2.1.
+
+---------------------------------------------------------------------------------
+
+This project include code from Google's Asylo project.
+
+* cpp/src/arrow/result.h is based on status_or.h
+
+Copyright (c) Copyright 2017 Asylo authors
+Homepage: https://asylo.dev/
+License: Apache 2.0
+
+--------------------------------------------------------------------------------
+
+This project includes code from Google's protobuf project
+
+* cpp/src/arrow/result.h ARROW_ASSIGN_OR_RAISE is based off ASSIGN_OR_RETURN
+
+Copyright 2008 Google Inc. All rights reserved.
+Homepage: https://developers.google.com/protocol-buffers/
+License:
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+ * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+ * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+ * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+Code generated by the Protocol Buffer compiler is owned by the owner
+of the input file used when generating it. This code is not
+standalone and requires a support library to be linked with it. This
+support library is itself covered by the above license.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency LLVM is statically linked in certain binary
+distributions. LLVM has the following license:
+
+==============================================================================
+LLVM Release License
+==============================================================================
+University of Illinois/NCSA
+Open Source License
+
+Copyright (c) 2003-2018 University of Illinois at Urbana-Champaign.
+All rights reserved.
+
+Developed by:
+
+ LLVM Team
+
+ University of Illinois at Urbana-Champaign
+
+ http://llvm.org
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal with
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
+of the Software, and to permit persons to whom the Software is furnished to do
+so, subject to the following conditions:
+
+ * Redistributions of source code must retain the above copyright notice,
+ this list of conditions and the following disclaimers.
+
+ * Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimers in the
+ documentation and/or other materials provided with the distribution.
+
+ * Neither the names of the LLVM Team, University of Illinois at
+ Urbana-Champaign, nor the names of its contributors may be used to
+ endorse or promote products derived from this Software without specific
+ prior written permission.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
+FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+CONTRIBUTORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS WITH THE
+SOFTWARE.
+
+==============================================================================
+Copyrights and Licenses for Third Party Software Distributed with LLVM:
+==============================================================================
+The LLVM software contains code written by third parties. Such software will
+have its own individual LICENSE.TXT file in the directory in which it appears.
+This file will describe the copyrights, license, and restrictions which apply
+to that code.
+
+The disclaimer of warranty in the University of Illinois Open Source License
+applies to all code in the LLVM Distribution, and nothing in any of the
+other licenses gives permission to use the names of the LLVM Team or the
+University of Illinois to endorse or promote products derived from this
+Software.
+
+The following pieces of software have additional or alternate copyrights,
+licenses, and/or restrictions:
+
+Program Directory
+------- ---------
+Google Test llvm/utils/unittest/googletest
+OpenBSD regex llvm/lib/Support/{reg*, COPYRIGHT.regex}
+pyyaml tests llvm/test/YAMLParser/{*.data, LICENSE.TXT}
+ARM contributions llvm/lib/Target/ARM/LICENSE.TXT
+md5 contributions llvm/lib/Support/MD5.cpp llvm/include/llvm/Support/MD5.h
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency gRPC is statically linked in certain binary
+distributions, like the python wheels. gRPC has the following license:
+
+Copyright 2014 gRPC authors.
+
+Licensed 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.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency Apache Thrift is statically linked in certain binary
+distributions, like the python wheels. Apache Thrift has the following license:
+
+Apache Thrift
+Copyright (C) 2006 - 2019, The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+Licensed 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.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency Apache ORC is statically linked in certain binary
+distributions, like the python wheels. Apache ORC has the following license:
+
+Apache ORC
+Copyright 2013-2019 The Apache Software Foundation
+
+This product includes software developed by The Apache Software
+Foundation (http://www.apache.org/).
+
+This product includes software developed by Hewlett-Packard:
+(c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P
+
+Licensed 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.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency zstd is statically linked in certain binary
+distributions, like the python wheels. ZSTD has the following license:
+
+BSD License
+
+For Zstandard software
+
+Copyright (c) 2016-present, Facebook, Inc. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+ * Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+ * Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+ * Neither the name Facebook nor the names of its contributors may be used to
+ endorse or promote products derived from this software without specific
+ prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency lz4 is statically linked in certain binary
+distributions, like the python wheels. lz4 has the following license:
+
+LZ4 Library
+Copyright (c) 2011-2016, Yann Collet
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice, this
+ list of conditions and the following disclaimer in the documentation and/or
+ other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency Brotli is statically linked in certain binary
+distributions, like the python wheels. Brotli has the following license:
+
+Copyright (c) 2009, 2010, 2013-2016 by the Brotli Authors.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency rapidjson is statically linked in certain binary
+distributions, like the python wheels. rapidjson and its dependencies have the
+following licenses:
+
+Tencent is pleased to support the open source community by making RapidJSON
+available.
+
+Copyright (C) 2015 THL A29 Limited, a Tencent company, and Milo Yip.
+All rights reserved.
+
+If you have downloaded a copy of the RapidJSON binary from Tencent, please note
+that the RapidJSON binary is licensed under the MIT License.
+If you have downloaded a copy of the RapidJSON source code from Tencent, please
+note that RapidJSON source code is licensed under the MIT License, except for
+the third-party components listed below which are subject to different license
+terms. Your integration of RapidJSON into your own projects may require
+compliance with the MIT License, as well as the other licenses applicable to
+the third-party components included within RapidJSON. To avoid the problematic
+JSON license in your own projects, it's sufficient to exclude the
+bin/jsonchecker/ directory, as it's the only code under the JSON license.
+A copy of the MIT License is included in this file.
+
+Other dependencies and licenses:
+
+ Open Source Software Licensed Under the BSD License:
+ --------------------------------------------------------------------
+
+ The msinttypes r29
+ Copyright (c) 2006-2013 Alexander Chemeris
+ All rights reserved.
+
+ Redistribution and use in source and binary forms, with or without
+ modification, are permitted provided that the following conditions are met:
+
+ * Redistributions of source code must retain the above copyright notice,
+ this list of conditions and the following disclaimer.
+ * Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+ * Neither the name of copyright holder nor the names of its contributors
+ may be used to endorse or promote products derived from this software
+ without specific prior written permission.
+
+ THIS SOFTWARE IS PROVIDED BY THE REGENTS AND CONTRIBUTORS ``AS IS'' AND ANY
+ EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+ WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+ DISCLAIMED. IN NO EVENT SHALL THE REGENTS AND CONTRIBUTORS BE LIABLE FOR
+ ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+ DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+ SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+ CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+ LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY
+ OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH
+ DAMAGE.
+
+ Open Source Software Licensed Under the JSON License:
+ --------------------------------------------------------------------
+
+ json.org
+ Copyright (c) 2002 JSON.org
+ All Rights Reserved.
+
+ JSON_checker
+ Copyright (c) 2002 JSON.org
+ All Rights Reserved.
+
+
+ Terms of the JSON License:
+ ---------------------------------------------------
+
+ Permission is hereby granted, free of charge, to any person obtaining a
+ copy of this software and associated documentation files (the "Software"),
+ to deal in the Software without restriction, including without limitation
+ the rights to use, copy, modify, merge, publish, distribute, sublicense,
+ and/or sell copies of the Software, and to permit persons to whom the
+ Software is furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in
+ all copies or substantial portions of the Software.
+
+ The Software shall be used for Good, not Evil.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+ FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
+ DEALINGS IN THE SOFTWARE.
+
+
+ Terms of the MIT License:
+ --------------------------------------------------------------------
+
+ Permission is hereby granted, free of charge, to any person obtaining a
+ copy of this software and associated documentation files (the "Software"),
+ to deal in the Software without restriction, including without limitation
+ the rights to use, copy, modify, merge, publish, distribute, sublicense,
+ and/or sell copies of the Software, and to permit persons to whom the
+ Software is furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included
+ in all copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+ FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
+ DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency snappy is statically linked in certain binary
+distributions, like the python wheels. snappy has the following license:
+
+Copyright 2011, Google Inc.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+ * Redistributions of source code must retain the above copyright notice,
+ this list of conditions and the following disclaimer.
+ * Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+ * Neither the name of Google Inc. nor the names of its contributors may be
+ used to endorse or promote products derived from this software without
+ specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+===
+
+Some of the benchmark data in testdata/ is licensed differently:
+
+ - fireworks.jpeg is Copyright 2013 Steinar H. Gunderson, and
+ is licensed under the Creative Commons Attribution 3.0 license
+ (CC-BY-3.0). See https://creativecommons.org/licenses/by/3.0/
+ for more information.
+
+ - kppkn.gtb is taken from the Gaviota chess tablebase set, and
+ is licensed under the MIT License. See
+ https://sites.google.com/site/gaviotachessengine/Home/endgame-tablebases-1
+ for more information.
+
+ - paper-100k.pdf is an excerpt (bytes 92160 to 194560) from the paper
+ “Combinatorial Modeling of Chromatin Features Quantitatively Predicts DNA
+ Replication Timing in _Drosophila_” by Federico Comoglio and Renato Paro,
+ which is licensed under the CC-BY license. See
+ http://www.ploscompbiol.org/static/license for more ifnormation.
+
+ - alice29.txt, asyoulik.txt, plrabn12.txt and lcet10.txt are from Project
+ Gutenberg. The first three have expired copyrights and are in the public
+ domain; the latter does not have expired copyright, but is still in the
+ public domain according to the license information
+ (http://www.gutenberg.org/ebooks/53).
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency gflags is statically linked in certain binary
+distributions, like the python wheels. gflags has the following license:
+
+Copyright (c) 2006, Google Inc.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+ * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+ * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+ * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency glog is statically linked in certain binary
+distributions, like the python wheels. glog has the following license:
+
+Copyright (c) 2008, Google Inc.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+ * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+ * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+ * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+A function gettimeofday in utilities.cc is based on
+
+http://www.google.com/codesearch/p?hl=en#dR3YEbitojA/COPYING&q=GetSystemTimeAsFileTime%20license:bsd
+
+The license of this code is:
+
+Copyright (c) 2003-2008, Jouni Malinen and contributors
+All Rights Reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+1. Redistributions of source code must retain the above copyright
+ notice, this list of conditions and the following disclaimer.
+
+2. Redistributions in binary form must reproduce the above copyright
+ notice, this list of conditions and the following disclaimer in the
+ documentation and/or other materials provided with the distribution.
+
+3. Neither the name(s) of the above-listed copyright holder(s) nor the
+ names of its contributors may be used to endorse or promote products
+ derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency re2 is statically linked in certain binary
+distributions, like the python wheels. re2 has the following license:
+
+Copyright (c) 2009 The RE2 Authors. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+ * Redistributions of source code must retain the above copyright
+ notice, this list of conditions and the following disclaimer.
+ * Redistributions in binary form must reproduce the above
+ copyright notice, this list of conditions and the following
+ disclaimer in the documentation and/or other materials provided
+ with the distribution.
+ * Neither the name of Google Inc. nor the names of its contributors
+ may be used to endorse or promote products derived from this
+ software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency c-ares is statically linked in certain binary
+distributions, like the python wheels. c-ares has the following license:
+
+# c-ares license
+
+Copyright (c) 2007 - 2018, Daniel Stenberg with many contributors, see AUTHORS
+file.
+
+Copyright 1998 by the Massachusetts Institute of Technology.
+
+Permission to use, copy, modify, and distribute this software and its
+documentation for any purpose and without fee is hereby granted, provided that
+the above copyright notice appear in all copies and that both that copyright
+notice and this permission notice appear in supporting documentation, and that
+the name of M.I.T. not be used in advertising or publicity pertaining to
+distribution of the software without specific, written prior permission.
+M.I.T. makes no representations about the suitability of this software for any
+purpose. It is provided "as is" without express or implied warranty.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency zlib is redistributed as a dynamically linked shared
+library in certain binary distributions, like the python wheels. In the future
+this will likely change to static linkage. zlib has the following license:
+
+zlib.h -- interface of the 'zlib' general purpose compression library
+ version 1.2.11, January 15th, 2017
+
+ Copyright (C) 1995-2017 Jean-loup Gailly and Mark Adler
+
+ This software is provided 'as-is', without any express or implied
+ warranty. In no event will the authors be held liable for any damages
+ arising from the use of this software.
+
+ Permission is granted to anyone to use this software for any purpose,
+ including commercial applications, and to alter it and redistribute it
+ freely, subject to the following restrictions:
+
+ 1. The origin of this software must not be misrepresented; you must not
+ claim that you wrote the original software. If you use this software
+ in a product, an acknowledgment in the product documentation would be
+ appreciated but is not required.
+ 2. Altered source versions must be plainly marked as such, and must not be
+ misrepresented as being the original software.
+ 3. This notice may not be removed or altered from any source distribution.
+
+ Jean-loup Gailly Mark Adler
+ jloup@gzip.org madler@alumni.caltech.edu
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency openssl is redistributed as a dynamically linked shared
+library in certain binary distributions, like the python wheels. openssl
+preceding version 3 has the following license:
+
+ LICENSE ISSUES
+ ==============
+
+ The OpenSSL toolkit stays under a double license, i.e. both the conditions of
+ the OpenSSL License and the original SSLeay license apply to the toolkit.
+ See below for the actual license texts.
+
+ OpenSSL License
+ ---------------
+
+/* ====================================================================
+ * Copyright (c) 1998-2019 The OpenSSL Project. All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ *
+ * 1. Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *
+ * 2. Redistributions in binary form must reproduce the above copyright
+ * notice, this list of conditions and the following disclaimer in
+ * the documentation and/or other materials provided with the
+ * distribution.
+ *
+ * 3. All advertising materials mentioning features or use of this
+ * software must display the following acknowledgment:
+ * "This product includes software developed by the OpenSSL Project
+ * for use in the OpenSSL Toolkit. (http://www.openssl.org/)"
+ *
+ * 4. The names "OpenSSL Toolkit" and "OpenSSL Project" must not be used to
+ * endorse or promote products derived from this software without
+ * prior written permission. For written permission, please contact
+ * openssl-core@openssl.org.
+ *
+ * 5. Products derived from this software may not be called "OpenSSL"
+ * nor may "OpenSSL" appear in their names without prior written
+ * permission of the OpenSSL Project.
+ *
+ * 6. Redistributions of any form whatsoever must retain the following
+ * acknowledgment:
+ * "This product includes software developed by the OpenSSL Project
+ * for use in the OpenSSL Toolkit (http://www.openssl.org/)"
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE OpenSSL PROJECT ``AS IS'' AND ANY
+ * EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
+ * PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE OpenSSL PROJECT OR
+ * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
+ * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+ * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
+ * OF THE POSSIBILITY OF SUCH DAMAGE.
+ * ====================================================================
+ *
+ * This product includes cryptographic software written by Eric Young
+ * (eay@cryptsoft.com). This product includes software written by Tim
+ * Hudson (tjh@cryptsoft.com).
+ *
+ */
+
+ Original SSLeay License
+ -----------------------
+
+/* Copyright (C) 1995-1998 Eric Young (eay@cryptsoft.com)
+ * All rights reserved.
+ *
+ * This package is an SSL implementation written
+ * by Eric Young (eay@cryptsoft.com).
+ * The implementation was written so as to conform with Netscapes SSL.
+ *
+ * This library is free for commercial and non-commercial use as long as
+ * the following conditions are aheared to. The following conditions
+ * apply to all code found in this distribution, be it the RC4, RSA,
+ * lhash, DES, etc., code; not just the SSL code. The SSL documentation
+ * included with this distribution is covered by the same copyright terms
+ * except that the holder is Tim Hudson (tjh@cryptsoft.com).
+ *
+ * Copyright remains Eric Young's, and as such any Copyright notices in
+ * the code are not to be removed.
+ * If this package is used in a product, Eric Young should be given attribution
+ * as the author of the parts of the library used.
+ * This can be in the form of a textual message at program startup or
+ * in documentation (online or textual) provided with the package.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the copyright
+ * notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ * notice, this list of conditions and the following disclaimer in the
+ * documentation and/or other materials provided with the distribution.
+ * 3. All advertising materials mentioning features or use of this software
+ * must display the following acknowledgement:
+ * "This product includes cryptographic software written by
+ * Eric Young (eay@cryptsoft.com)"
+ * The word 'cryptographic' can be left out if the rouines from the library
+ * being used are not cryptographic related :-).
+ * 4. If you include any Windows specific code (or a derivative thereof) from
+ * the apps directory (application code) you must include an acknowledgement:
+ * "This product includes software written by Tim Hudson (tjh@cryptsoft.com)"
+ *
+ * THIS SOFTWARE IS PROVIDED BY ERIC YOUNG ``AS IS'' AND
+ * ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ * ARE DISCLAIMED. IN NO EVENT SHALL THE AUTHOR OR CONTRIBUTORS BE LIABLE
+ * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+ * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS
+ * OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
+ * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY
+ * OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
+ * SUCH DAMAGE.
+ *
+ * The licence and distribution terms for any publically available version or
+ * derivative of this code cannot be changed. i.e. this code cannot simply be
+ * copied and put under another distribution licence
+ * [including the GNU Public Licence.]
+ */
+
+--------------------------------------------------------------------------------
+
+This project includes code from the rtools-backports project.
+
+* ci/scripts/PKGBUILD and ci/scripts/r_windows_build.sh are based on code
+ from the rtools-backports project.
+
+Copyright: Copyright (c) 2013 - 2019, Алексей and Jeroen Ooms.
+All rights reserved.
+Homepage: https://github.com/r-windows/rtools-backports
+License: 3-clause BSD
+
+--------------------------------------------------------------------------------
+
+Some code from pandas has been adapted for the pyarrow codebase. pandas is
+available under the 3-clause BSD license, which follows:
+
+pandas license
+==============
+
+Copyright (c) 2011-2012, Lambda Foundry, Inc. and PyData Development Team
+All rights reserved.
+
+Copyright (c) 2008-2011 AQR Capital Management, LLC
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+ * Redistributions of source code must retain the above copyright
+ notice, this list of conditions and the following disclaimer.
+
+ * Redistributions in binary form must reproduce the above
+ copyright notice, this list of conditions and the following
+ disclaimer in the documentation and/or other materials provided
+ with the distribution.
+
+ * Neither the name of the copyright holder nor the names of any
+ contributors may be used to endorse or promote products derived
+ from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDER AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+Some bits from DyND, in particular aspects of the build system, have been
+adapted from libdynd and dynd-python under the terms of the BSD 2-clause
+license
+
+The BSD 2-Clause License
+
+ Copyright (C) 2011-12, Dynamic NDArray Developers
+ All rights reserved.
+
+ Redistribution and use in source and binary forms, with or without
+ modification, are permitted provided that the following conditions are
+ met:
+
+ * Redistributions of source code must retain the above copyright
+ notice, this list of conditions and the following disclaimer.
+
+ * Redistributions in binary form must reproduce the above
+ copyright notice, this list of conditions and the following
+ disclaimer in the documentation and/or other materials provided
+ with the distribution.
+
+ THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+Dynamic NDArray Developers list:
+
+ * Mark Wiebe
+ * Continuum Analytics
+
+--------------------------------------------------------------------------------
+
+Some source code from Ibis (https://github.com/cloudera/ibis) has been adapted
+for PyArrow. Ibis is released under the Apache License, Version 2.0.
+
+--------------------------------------------------------------------------------
+
+This project includes code from the autobrew project.
+
+* r/tools/autobrew and dev/tasks/homebrew-formulae/autobrew/apache-arrow.rb
+ are based on code from the autobrew project.
+
+Copyright (c) 2019, Jeroen Ooms
+License: MIT
+Homepage: https://github.com/jeroen/autobrew
+
+--------------------------------------------------------------------------------
+
+dev/tasks/homebrew-formulae/apache-arrow.rb has the following license:
+
+BSD 2-Clause License
+
+Copyright (c) 2009-present, Homebrew contributors
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+----------------------------------------------------------------------
+
+cpp/src/arrow/vendored/base64.cpp has the following license
+
+ZLIB License
+
+Copyright (C) 2004-2017 René Nyffenegger
+
+This source code is provided 'as-is', without any express or implied
+warranty. In no event will the author be held liable for any damages arising
+from the use of this software.
+
+Permission is granted to anyone to use this software for any purpose, including
+commercial applications, and to alter it and redistribute it freely, subject to
+the following restrictions:
+
+1. The origin of this source code must not be misrepresented; you must not
+ claim that you wrote the original source code. If you use this source code
+ in a product, an acknowledgment in the product documentation would be
+ appreciated but is not required.
+
+2. Altered source versions must be plainly marked as such, and must not be
+ misrepresented as being the original source code.
+
+3. This notice may not be removed or altered from any source distribution.
+
+René Nyffenegger rene.nyffenegger@adp-gmbh.ch
+
+--------------------------------------------------------------------------------
+
+The file cpp/src/arrow/vendored/optional.hpp has the following license
+
+Boost Software License - Version 1.0 - August 17th, 2003
+
+Permission is hereby granted, free of charge, to any person or organization
+obtaining a copy of the software and accompanying documentation covered by
+this license (the "Software") to use, reproduce, display, distribute,
+execute, and transmit the Software, and to prepare derivative works of the
+Software, and to permit third-parties to whom the Software is furnished to
+do so, all subject to the following:
+
+The copyright notices in the Software and this entire statement, including
+the above license grant, this restriction and the following disclaimer,
+must be included in all copies of the Software, in whole or in part, and
+all derivative works of the Software, unless such copies or derivative
+works are solely in the form of machine-executable object code generated by
+a source language processor.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT
+SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE
+FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE,
+ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
+DEALINGS IN THE SOFTWARE.