diff --git a/.gitignore b/.gitignore new file mode 100644 index 000000000..4eaf66513 --- /dev/null +++ b/.gitignore @@ -0,0 +1,10 @@ +target +.idea +*.iml +derby.log +metastore_db/ +spark-warehouse/ +dependency-reduced-pom.xml +core/src/execution/generated +prebuild +.flattened-pom.xml diff --git a/.scalafix.conf b/.scalafix.conf new file mode 100644 index 000000000..0f23f57cd --- /dev/null +++ b/.scalafix.conf @@ -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. +rules = [ + ExplicitResultTypes, + NoAutoTupling, + RemoveUnused, + + DisableSyntax, + LeakingImplicitClassVal, + NoValInForComprehension, + ProcedureSyntax, + RedundantSyntax +] diff --git a/DEBUGGING.md b/DEBUGGING.md new file mode 100644 index 000000000..e348b7215 --- /dev/null +++ b/DEBUGGING.md @@ -0,0 +1,96 @@ + + +# Comet Debugging Guide + +This HOWTO describes how to debug JVM code and Native code concurrently. The guide assumes you have: +1. Intellij as the Java IDE +2. CLion as the Native IDE. For Rust code, the CLion Rust language plugin is required. Note that the +Intellij Rust plugin is not sufficient. +3. CLion/LLDB as the native debugger. CLion ships with a bundled LLDB and the Rust community has +its own packaging of LLDB (`lldb-rust`). Both provide a better display of Rust symbols than plain +LLDB or the LLDB that is bundled with XCode. We will use the LLDB packaged with CLion for this guide. +4. We will use a Comet _unit_ test as the canonical use case. + +_Caveat: The steps here have only been tested with JDK 11_ on Mac (M1) + +## Debugging for Advanced Developers + +Add a `.lldbinit` to comet/core. This is not strictly necessary but will be useful if you want to +use advanced `lldb` debugging. + +### In Intellij + +1. Set a breakpoint in `NativeBase.load()`, at a point _after_ the Comet library has been loaded. + +1. Add a Debug Configuration for the unit test + +1. In the Debug Configuration for that unit test add `-Xint` as a JVM parameter. This option is +undocumented *magic*. Without this, the LLDB debugger hits a EXC_BAD_ACCESS (or EXC_BAD_INSTRUCTION) from +which one cannot recover. + +1. Add a println to the unit test to print the PID of the JVM process. (jps can also be used but this is less error prone if you have multiple jvm processes running) + ``` JDK8 + println("Waiting for Debugger: PID - ", ManagementFactory.getRuntimeMXBean().getName()) + ``` + This will print something like : `PID@your_machine_name`. + + For JDK9 and newer + ```JDK9 + println("Waiting for Debugger: PID - ", ProcessHandle.current.pid) + ``` + + ==> Note the PID + +1. Debug-run the test in Intellij and wait for the breakpoint to be hit + +### In CLion + +1. After the breakpoint is hit in Intellij, in Clion (or LLDB from terminal or editor) - + + 1. Attach to the jvm process (make sure the PID matches). In CLion, this is `Run -> Atttach to process` + + 1. Put your breakpoint in the native code + +1. Go back to intellij and resume the process. + +1. Most debugging in CLion is similar to Intellij. For advanced LLDB based debugging the LLDB command line can be accessed from the LLDB tab in the Debugger view. Refer to the [LLDB manual](https://lldb.llvm.org/use/tutorial.html) for LLDB commands. + +### After your debugging is done, + +1. In CLion, detach from the process if not already detached + +2. In Intellij, the debugger might have lost track of the process. If so, the debugger tab + will show the process as running (even if the test/job is shown as completed). + +3. Close the debugger tab, and if the IDS asks whether it should terminate the process, + click Yes. + +4. In terminal, use jps to identify the process with the process id you were debugging. If + it shows up as running, kill -9 [pid]. If that doesn't remove the process, don't bother, + the process will be left behind as a zombie and will consume no (significant) resources. + Eventually it will be cleaned up when you reboot possibly after a software update. + +### Additional Info + +OpenJDK mailing list on debugging the JDK on MacOS +https://mail.openjdk.org/pipermail/hotspot-dev/2019-September/039429.html + +Detecting the debugger +https://stackoverflow.com/questions/5393403/can-a-java-application-detect-that-a-debugger-is-attached#:~:text=No.,to%20let%20your%20app%20continue.&text=I%20know%20that%20those%20are,meant%20with%20my%20first%20phrase). diff --git a/DEVELOPMENT.md b/DEVELOPMENT.md new file mode 100644 index 000000000..1793bb9e2 --- /dev/null +++ b/DEVELOPMENT.md @@ -0,0 +1,65 @@ + + +# Comet Development Guide + +## Project Layout + +``` +├── common <- common Java/Scala code +├── conf <- configuration files +├── core <- core native code, in Rust +├── spark <- Spark integration +``` + +## Development Setup + +1. Make sure `JAVA_HOME` is set and point to JDK 11 installation. +2. Install Rust toolchain. The easiest way is to use + [rustup](https://rustup.rs). + +## Build & Test + +A few common commands are specified in project's `Makefile`: + +- `make`: compile the entire project, but don't run tests +- `make test`: compile the project and run tests in both Rust and Java + side. +- `make release`: compile the project and creates a release build. This + is useful when you want to test Comet local installation in another project + such as Spark. +- `make clean`: clean up the workspace +- `bin/comet-spark-shell -d . -o spark/target/` run Comet spark shell for V1 datasources +- `bin/comet-spark-shell -d . -o spark/target/ --conf spark.sql.sources.useV1SourceList=""` run Comet spark shell for V2 datasources + +## Benchmark + +There's a `make` command to run micro benchmarks in the repo. For +instance: + +``` +make benchmark-org.apache.spark.sql.benchmark.CometReadBenchmark +``` + +To run TPC-H or TPC-DS micro benchmarks, please follow the instructions +in the respective source code, e.g., `CometTPCHQueryBenchmark`. + +## Debugging +Comet is a multi-language project with native code written in Rust and JVM code written in Java and Scala. +It is possible to debug both native and JVM code concurrently as described in the [DEBUGGING guide](DEBUGGING.md) diff --git a/EXPRESSIONS.md b/EXPRESSIONS.md new file mode 100644 index 000000000..40e140aba --- /dev/null +++ b/EXPRESSIONS.md @@ -0,0 +1,96 @@ + + +# Expressions Supported by Comet + +The following Spark expressions are currently available: + ++ Literals ++ Arithmetic Operators + + UnaryMinus + + Add/Minus/Multiply/Divide/Remainder ++ Conditional functions + + Case When + + If ++ Cast ++ Coalesce ++ Boolean functions + + And + + Or + + Not + + EqualTo + + EqualNullSafe + + GreaterThan + + GreaterThanOrEqual + + LessThan + + LessThanOrEqual + + IsNull + + IsNotNull + + In ++ String functions + + Substring + + Coalesce + + StringSpace + + Like + + Contains + + Startswith + + Endswith + + Ascii + + Bit_length + + Octet_length + + Upper + + Lower + + Chr + + Initcap + + Trim/Btrim/Ltrim/Rtrim + + Concat_ws + + Repeat + + Length + + Reverse + + Instr + + Replace + + Translate ++ Bitwise functions + + Shiftright/Shiftleft ++ Date/Time functions + + Year/Hour/Minute/Second ++ Math functions + + Abs + + Acos + + Asin + + Atan + + Atan2 + + Cos + + Exp + + Ln + + Log10 + + Log2 + + Pow + + Round + + Signum + + Sin + + Sqrt + + Tan + + Ceil + + Floor ++ Aggregate functions + + Count + + Sum + + Max + + Min diff --git a/LICENSE.txt b/LICENSE.txt new file mode 100644 index 000000000..d74c6b599 --- /dev/null +++ b/LICENSE.txt @@ -0,0 +1,212 @@ + + 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 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 diff --git a/Makefile b/Makefile new file mode 100644 index 000000000..475c88736 --- /dev/null +++ b/Makefile @@ -0,0 +1,85 @@ +# 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. + +.PHONY: all core jvm test clean release-linux release bench + +all: core jvm + +core: + cd core && cargo build +jvm: + mvn clean package -DskipTests $(PROFILES) +test: + mvn clean + # We need to compile CometException so that the cargo test can pass + mvn compile -pl common -DskipTests $(PROFILES) + cd core && cargo build && \ + LD_LIBRARY_PATH=${LD_LIBRARY_PATH:+${LD_LIBRARY_PATH}:}${JAVA_HOME}/lib:${JAVA_HOME}/lib/server:${JAVA_HOME}/lib/jli && \ + DYLD_LIBRARY_PATH=${DYLD_LIBRARY_PATH:+${DYLD_LIBRARY_PATH}:}${JAVA_HOME}/lib:${JAVA_HOME}/lib/server:${JAVA_HOME}/lib/jli \ + RUST_BACKTRACE=1 cargo test + SPARK_HOME=`pwd` COMET_CONF_DIR=$(shell pwd)/conf RUST_BACKTRACE=1 mvn verify $(PROFILES) +clean: + cd core && cargo clean + mvn clean + rm -rf .dist +bench: + cd core && LD_LIBRARY_PATH=${LD_LIBRARY_PATH:+${LD_LIBRARY_PATH}:}${JAVA_HOME}/lib:${JAVA_HOME}/lib/server:${JAVA_HOME}/lib/jli && \ + DYLD_LIBRARY_PATH=${DYLD_LIBRARY_PATH:+${DYLD_LIBRARY_PATH}:}${JAVA_HOME}/lib:${JAVA_HOME}/lib/server:${JAVA_HOME}/lib/jli \ + RUSTFLAGS="-Ctarget-cpu=native" cargo bench $(filter-out $@,$(MAKECMDGOALS)) +format: + mvn compile test-compile scalafix:scalafix -Psemanticdb $(PROFILES) + mvn spotless:apply $(PROFILES) + +core-amd64: + rustup target add x86_64-apple-darwin + cd core && RUSTFLAGS="-Ctarget-cpu=skylake -Ctarget-feature=-prefer-256-bit" CC=o64-clang CXX=o64-clang++ cargo build --target x86_64-apple-darwin --features nightly --release + mkdir -p common/target/classes/org/apache/comet/darwin/x86_64 + cp core/target/x86_64-apple-darwin/release/libcomet.dylib common/target/classes/org/apache/comet/darwin/x86_64 + cd core && RUSTFLAGS="-Ctarget-cpu=haswell -Ctarget-feature=-prefer-256-bit" cargo build --features nightly --release + mkdir -p common/target/classes/org/apache/comet/linux/amd64 + cp core/target/release/libcomet.so common/target/classes/org/apache/comet/linux/amd64 + jar -cf common/target/comet-native-x86_64.jar \ + -C common/target/classes/org/apache/comet darwin \ + -C common/target/classes/org/apache/comet linux + ./dev/deploy-file common/target/comet-native-x86_64.jar comet-native-x86_64${COMET_CLASSIFIER} jar + +core-arm64: + rustup target add aarch64-apple-darwin + cd core && RUSTFLAGS="-Ctarget-cpu=apple-m1" CC=arm64-apple-darwin21.4-clang CXX=arm64-apple-darwin21.4-clang++ CARGO_FEATURE_NEON=1 cargo build --target aarch64-apple-darwin --features nightly --release + mkdir -p common/target/classes/org/apache/comet/darwin/aarch64 + cp core/target/aarch64-apple-darwin/release/libcomet.dylib common/target/classes/org/apache/comet/darwin/aarch64 + cd core && RUSTFLAGS="-Ctarget-cpu=native" cargo build --features nightly --release + mkdir -p common/target/classes/org/apache/comet/linux/aarch64 + cp core/target/release/libcomet.so common/target/classes/org/apache/comet/linux/aarch64 + jar -cf common/target/comet-native-aarch64.jar \ + -C common/target/classes/org/apache/comet darwin \ + -C common/target/classes/org/apache/comet linux + ./dev/deploy-file common/target/comet-native-aarch64.jar comet-native-aarch64${COMET_CLASSIFIER} jar + +release-linux: clean + rustup target add aarch64-apple-darwin x86_64-apple-darwin + cd core && RUSTFLAGS="-Ctarget-cpu=apple-m1" CC=arm64-apple-darwin21.4-clang CXX=arm64-apple-darwin21.4-clang++ CARGO_FEATURE_NEON=1 cargo build --target aarch64-apple-darwin --features nightly --release + cd core && RUSTFLAGS="-Ctarget-cpu=skylake -Ctarget-feature=-prefer-256-bit" CC=o64-clang CXX=o64-clang++ cargo build --target x86_64-apple-darwin --features nightly --release + cd core && RUSTFLAGS="-Ctarget-cpu=native -Ctarget-feature=-prefer-256-bit" cargo build --features nightly --release + mvn install -Prelease -DskipTests $(PROFILES) +release: + cd core && RUSTFLAGS="-Ctarget-cpu=native" cargo build --features nightly --release + mvn install -Prelease -DskipTests $(PROFILES) +benchmark-%: clean release + cd spark && COMET_CONF_DIR=$(shell pwd)/conf MAVEN_OPTS='-Xmx20g' .mvn exec:java -Dexec.mainClass="$*" -Dexec.classpathScope="test" -Dexec.cleanupDaemonThreads="false" -Dexec.args="$(filter-out $@,$(MAKECMDGOALS))" $(PROFILES) +.DEFAULT: + @: # ignore arguments provided to benchmarks e.g. "make benchmark-foo -- --bar", we do not want to treat "--bar" as target diff --git a/README.md b/README.md index ba486a23f..ba50f673a 100644 --- a/README.md +++ b/README.md @@ -1 +1,60 @@ -# arrow-datafusion-comet + + +# Apache Arrow DataFusion Comet + +Comet is an Apache Spark plugin that uses [Apache Arrow DataFusion](https://arrow.apache.org/datafusion/) +as native runtime to achieve improvement in terms of query efficiency and query runtime. + +On a high level, Comet aims to support: +- a native Parquet implementation, including both reader and writer +- full implementation of Spark operators, including + Filter/Project/Aggregation/Join/Exchange etc. +- full implementation of Spark built-in expressions +- a UDF framework for users to migrate their existing UDF to native + +The following diagram illustrates the architecture of Comet: + + + +## Current Status + +The project is currently integrated into Apache Spark 3.2, 3.3, and 3.4. + +## Feature Parity with Apache Spark + +The project strives to keep feature parity with Apache Spark, that is, +users should expect the same behavior (w.r.t features, configurations, +query results, etc) with Comet turned on or turned off in their Spark +jobs. In addition, Comet extension should automatically detect unsupported +features and fallback to Spark engine. + +To achieve this, besides unit tests within Comet itself, we also re-use +Spark SQL tests and make sure they all pass with Comet extension +enabled. + +## Supported Platforms + +Linux, Apple OSX (Intel and M1) + +## Requirements + +- Apache Spark 3.2, 3.3, or 3.4 +- JDK 8 and up +- GLIBC 2.17 (Centos 7) and up diff --git a/bin/comet-spark-shell b/bin/comet-spark-shell new file mode 100755 index 000000000..9ae55a1b3 --- /dev/null +++ b/bin/comet-spark-shell @@ -0,0 +1,84 @@ +#!/bin/bash + +# 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. + + +set -e + +if [[ -z ${SPARK_HOME} ]]; then + echo "ERROR: SPARK_HOME is not set: please set it to point to a Spark 3.4.0 distribution" + exit 1 +fi + +POSITIONAL=() +while [[ $# -gt 0 ]]; do + key="$1" + + case $key in + -r|--rebuild) + REBUILD=YES + shift + ;; + -o|--outdir) + COMET_OUTDIR=$2 + shift + shift + ;; + -d|--comet-dir) + COMET_DIR=$2 + shift + shift + ;; + *) + POSITIONAL+=("$1") + shift + ;; + esac +done + +set -- "${POSITIONAL[@]}" + +COMET_DIR="${COMET_DIR:-$HOME/git/comet}" +SPARK_VERSION_SHORT=3.4 +SCALA_BINARY_VERSION=2.12 +COMET_VERSION=0.1.0-SNAPSHOT +COMET_SPARK_JAR=comet-spark-spark${SPARK_VERSION_SHORT}_${SCALA_BINARY_VERSION}-${COMET_VERSION}.jar +COMET_OUTDIR="${COMET_OUTDIR:-/tmp}" + +if [[ ! -d $COMET_DIR ]]; then + echo "Comet repo: $COMET_DIR, doesn't exist" + exit 1 +fi + +if [[ ! -d $COMET_OUTDIR ]]; then + echo "Output directory for Comet Spark library: $COMET_OUTDIR, doesn't exist" + exit 1 +fi + +if [[ "X$REBUILD" == "XYES" ]]; then + cd $COMET_DIR && make release + cd $COMET_DIR/spark && cp target/${COMET_SPARK_JAR} $COMET_OUTDIR/${COMET_SPARK_JAR} +fi + +RUST_BACKTRACE=1 $SPARK_HOME/bin/spark-shell \ + --jars $COMET_OUTDIR/${COMET_SPARK_JAR} \ + --conf spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions \ + --conf spark.comet.enabled=true \ + --conf spark.comet.exec.enabled=true \ + --conf spark.comet.exec.all.enabled=true \ +$@ diff --git a/common/pom.xml b/common/pom.xml new file mode 100644 index 000000000..f885c346d --- /dev/null +++ b/common/pom.xml @@ -0,0 +1,200 @@ + + + + + + + 4.0.0 + + org.apache.comet + comet-parent-spark${spark.version.short}_${scala.binary.version} + 0.1.0-SNAPSHOT + ../pom.xml + + + comet-common-spark${spark.version.short}_${scala.binary.version} + comet-common + + + + false + + + + + org.apache.spark + spark-sql_${scala.binary.version} + + + org.apache.parquet + parquet-column + + + org.apache.parquet + parquet-hadoop + + + org.apache.arrow + arrow-vector + + + org.apache.arrow + arrow-memory-unsafe + + + org.apache.arrow + arrow-c-data + + + junit + junit + test + + + org.assertj + assertj-core + test + + + + + + + io.github.git-commit-id + git-commit-id-maven-plugin + 5.0.0 + + + get-the-git-infos + + revision + + initialize + + + + true + ${project.build.outputDirectory}/comet-git-info.properties + full + + ^git.branch$ + ^git.build.*$ + ^git.commit.id.(abbrev|full)$ + ^git.remote.*$ + + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + true + true + false + true + + + + org.apache.arrow:* + + + + + *:* + + **/*.thrift + git.properties + log4j.properties + log4j2.properties + arrow-git.properties + + + + org.apache.arrow:arrow-vector + + + codegen/** + + + + + + org.apache.arrow + ${comet.shade.packageName}.arrow + + + org/apache/arrow/c/jni/JniWrapper + org/apache/arrow/c/jni/PrivateData + org/apache/arrow/c/jni/CDataJniException + + org/apache/arrow/c/ArrayStreamExporter$ExportedArrayStreamPrivateData + + + + + + + + + + net.alchim31.maven + scala-maven-plugin + + + + + ${project.basedir}/src/main/resources + + + ${project.basedir}/../core/target/x86_64-apple-darwin/release + + libcomet.dylib + + org/apache/comet/darwin/x86_64 + + + ${project.basedir}/../core/target/aarch64-apple-darwin/release + + libcomet.dylib + + org/apache/comet/darwin/aarch64 + + + ${jni.dir} + + libcomet.dylib + libcomet.so + + org/apache/comet/${platform}/${arch} + + + + + diff --git a/common/src/main/java/org/apache/arrow/c/ArrowImporter.java b/common/src/main/java/org/apache/arrow/c/ArrowImporter.java new file mode 100644 index 000000000..90398cb72 --- /dev/null +++ b/common/src/main/java/org/apache/arrow/c/ArrowImporter.java @@ -0,0 +1,61 @@ +/* + * 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.arrow.c; + +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.types.pojo.Field; + +/** + * This class is used to import Arrow schema and array from native execution/shuffle. We cannot use + * Arrow's Java API to import schema and array directly because Arrow's Java API `Data.importField` + * initiates a new `SchemaImporter` for each field. Each `SchemaImporter` maintains an internal + * dictionary id counter. So the dictionary ids for multiple dictionary columns will conflict with + * each other and cause data corruption. + */ +public class ArrowImporter { + private final SchemaImporter importer; + private final BufferAllocator allocator; + + public ArrowImporter(BufferAllocator allocator) { + this.allocator = allocator; + this.importer = new SchemaImporter(allocator); + } + + Field importField(ArrowSchema schema, CDataDictionaryProvider provider) { + Field var4; + try { + var4 = importer.importField(schema, provider); + } finally { + schema.release(); + schema.close(); + } + + return var4; + } + + public FieldVector importVector( + ArrowArray array, ArrowSchema schema, CDataDictionaryProvider provider) { + Field field = importField(schema, provider); + FieldVector vector = field.createVector(allocator); + Data.importIntoVector(allocator, array, vector, provider); + return vector; + } +} diff --git a/common/src/main/java/org/apache/comet/CometRuntimeException.java b/common/src/main/java/org/apache/comet/CometRuntimeException.java new file mode 100644 index 000000000..b136ad165 --- /dev/null +++ b/common/src/main/java/org/apache/comet/CometRuntimeException.java @@ -0,0 +1,31 @@ +/* + * 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.comet; + +/** The parent class for all Comet runtime exceptions */ +public class CometRuntimeException extends RuntimeException { + public CometRuntimeException(String message) { + super(message); + } + + public CometRuntimeException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/common/src/main/java/org/apache/comet/NativeBase.java b/common/src/main/java/org/apache/comet/NativeBase.java new file mode 100644 index 000000000..42357b9bf --- /dev/null +++ b/common/src/main/java/org/apache/comet/NativeBase.java @@ -0,0 +1,278 @@ +/* + * 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.comet; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FilenameFilter; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.nio.file.Files; +import java.nio.file.StandardCopyOption; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.sql.comet.util.Utils; + +import static org.apache.comet.Constants.LOG_CONF_NAME; +import static org.apache.comet.Constants.LOG_CONF_PATH; + +/** Base class for JNI bindings. MUST be inherited by all classes that introduce JNI APIs. */ +public abstract class NativeBase { + static final String ARROW_UNSAFE_MEMORY_ACCESS = "arrow.enable_unsafe_memory_access"; + static final String ARROW_NULL_CHECK_FOR_GET = "arrow.enable_null_check_for_get"; + + private static final Logger LOG = LoggerFactory.getLogger(NativeBase.class); + private static final String NATIVE_LIB_NAME = "comet"; + + private static final String libraryToLoad = System.mapLibraryName(NATIVE_LIB_NAME); + private static boolean loaded = false; + private static final String searchPattern = "libcomet-"; + + static { + if (!isLoaded()) { + load(); + } + } + + public static synchronized boolean isLoaded() { + return loaded; + } + + // Only for testing + static synchronized void setLoaded(boolean b) { + loaded = b; + } + + static synchronized void load() { + if (loaded) { + return; + } + + cleanupOldTempLibs(); + + // Check if the arch used by JDK is the same as arch on the host machine, in particular, + // whether x86_64 JDK is used in arm64 Mac + if (!checkArch()) { + LOG.warn( + "Comet is disabled. JDK compiled for x86_64 is used in a Mac based on Apple Silicon. " + + "In order to use Comet, Please install a JDK version for ARM64 architecture"); + return; + } + + // Try to load Comet library from the java.library.path. + try { + System.loadLibrary(libraryToLoad); + loaded = true; + } catch (UnsatisfiedLinkError ex) { + // Doesn't exist, so proceed to loading bundled library. + bundleLoadLibrary(); + } + + initWithLogConf(); + // Only set the Arrow properties when debugging mode is off + if (!(boolean) CometConf.COMET_DEBUG_ENABLED().get()) { + setArrowProperties(); + } + } + + /** + * Use the bundled native libraries. Functionally equivalent to System.loadLibrary. + */ + private static void bundleLoadLibrary() { + String resourceName = resourceName(); + InputStream is = NativeBase.class.getResourceAsStream(resourceName); + if (is == null) { + throw new UnsupportedOperationException( + "Unsupported OS/arch, cannot find " + + resourceName + + ". Please try building from source."); + } + + File tempLib = null; + File tempLibLock = null; + try { + // Create the .lck file first to avoid a race condition + // with other concurrently running Java processes using Comet. + tempLibLock = File.createTempFile(searchPattern, "." + os().libExtension + ".lck"); + tempLib = new File(tempLibLock.getAbsolutePath().replaceFirst(".lck$", "")); + // copy to tempLib + Files.copy(is, tempLib.toPath(), StandardCopyOption.REPLACE_EXISTING); + System.load(tempLib.getAbsolutePath()); + loaded = true; + } catch (IOException e) { + throw new IllegalStateException("Cannot unpack libcomet: " + e); + } finally { + if (!loaded) { + if (tempLib != null && tempLib.exists()) { + if (!tempLib.delete()) { + LOG.error( + "Cannot unpack libcomet / cannot delete a temporary native library " + tempLib); + } + } + if (tempLibLock != null && tempLibLock.exists()) { + if (!tempLibLock.delete()) { + LOG.error( + "Cannot unpack libcomet / cannot delete a temporary lock file " + tempLibLock); + } + } + } else { + tempLib.deleteOnExit(); + tempLibLock.deleteOnExit(); + } + } + } + + private static void initWithLogConf() { + String logConfPath = System.getProperty(LOG_CONF_PATH(), Utils.getConfPath(LOG_CONF_NAME())); + + // If both the system property and the environmental variable failed to find a log + // configuration, then fall back to using the deployed default + if (logConfPath == null) { + LOG.info( + "Couldn't locate log file from either COMET_CONF_DIR or comet.log.file.path. " + + "Using default log configuration which emits to stdout"); + logConfPath = ""; + } else { + LOG.info("Using {} for native library logging", logConfPath); + } + init(logConfPath); + } + + private static void cleanupOldTempLibs() { + String tempFolder = new File(System.getProperty("java.io.tmpdir")).getAbsolutePath(); + File dir = new File(tempFolder); + + File[] tempLibFiles = + dir.listFiles( + new FilenameFilter() { + public boolean accept(File dir, String name) { + return name.startsWith(searchPattern) && !name.endsWith(".lck"); + } + }); + + if (tempLibFiles != null) { + for (File tempLibFile : tempLibFiles) { + File lckFile = new File(tempLibFile.getAbsolutePath() + ".lck"); + if (!lckFile.exists()) { + try { + tempLibFile.delete(); + } catch (SecurityException e) { + LOG.error("Failed to delete old temp lib", e); + } + } + } + } + } + + // Set Arrow related properties upon initializing native, such as enabling unsafe memory access + // as well as disabling null check for get, for performance reasons. + private static void setArrowProperties() { + setPropertyIfNull(ARROW_UNSAFE_MEMORY_ACCESS, "true"); + setPropertyIfNull(ARROW_NULL_CHECK_FOR_GET, "false"); + } + + private static void setPropertyIfNull(String key, String value) { + if (System.getProperty(key) == null) { + LOG.info("Setting system property {} to {}", key, value); + System.setProperty(key, value); + } else { + LOG.info( + "Skip setting system property {} to {}, because it is already set to {}", + key, + value, + System.getProperty(key)); + } + } + + private enum OS { + // Even on Windows, the default compiler from cpptasks (gcc) uses .so as a shared lib extension + WINDOWS("win32", "so"), + LINUX("linux", "so"), + MAC("darwin", "dylib"), + SOLARIS("solaris", "so"); + public final String name, libExtension; + + OS(String name, String libExtension) { + this.name = name; + this.libExtension = libExtension; + } + } + + private static String arch() { + return System.getProperty("os.arch"); + } + + private static OS os() { + String osName = System.getProperty("os.name"); + if (osName.contains("Linux")) { + return OS.LINUX; + } else if (osName.contains("Mac")) { + return OS.MAC; + } else if (osName.contains("Windows")) { + return OS.WINDOWS; + } else if (osName.contains("Solaris") || osName.contains("SunOS")) { + return OS.SOLARIS; + } else { + throw new UnsupportedOperationException("Unsupported operating system: " + osName); + } + } + + // For some reason users will get JVM crash when running Comet that is compiled for `aarch64` + // using a JVM that is compiled against `amd64`. Here we check if that is the case and fallback + // to Spark accordingly. + private static boolean checkArch() { + if (os() == OS.MAC) { + try { + String javaArch = arch(); + Process process = Runtime.getRuntime().exec("uname -a"); + if (process.waitFor() == 0) { + BufferedReader in = new BufferedReader(new InputStreamReader(process.getInputStream())); + String line; + while ((line = in.readLine()) != null) { + if (javaArch.equals("x86_64") && line.contains("ARM64")) { + return false; + } + } + } + } catch (IOException | InterruptedException e) { + LOG.warn("Error parsing host architecture", e); + } + } + + return true; + } + + private static String resourceName() { + OS os = os(); + String packagePrefix = NativeBase.class.getPackage().getName().replace('.', '/'); + + return "/" + packagePrefix + "/" + os.name + "/" + arch() + "/" + libraryToLoad; + } + + /** + * Initialize the native library through JNI. + * + * @param logConfPath location to the native log configuration file + */ + static native void init(String logConfPath); +} diff --git a/common/src/main/java/org/apache/comet/ParquetRuntimeException.java b/common/src/main/java/org/apache/comet/ParquetRuntimeException.java new file mode 100644 index 000000000..1f81587e7 --- /dev/null +++ b/common/src/main/java/org/apache/comet/ParquetRuntimeException.java @@ -0,0 +1,31 @@ +/* + * 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.comet; + +/** The parent class for the subset of Comet runtime exceptions related to Parquet. */ +public class ParquetRuntimeException extends CometRuntimeException { + public ParquetRuntimeException(String message) { + super(message); + } + + public ParquetRuntimeException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java b/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java new file mode 100644 index 000000000..099c7b973 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java @@ -0,0 +1,116 @@ +/* + * 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.comet.parquet; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.TimestampNTZType$; + +import org.apache.comet.CometConf; +import org.apache.comet.vector.CometVector; + +/** Base class for Comet Parquet column reader implementations. */ +public abstract class AbstractColumnReader implements AutoCloseable { + protected static final Logger LOG = LoggerFactory.getLogger(AbstractColumnReader.class); + + /** The Spark data type. */ + protected final DataType type; + + /** Parquet column descriptor. */ + protected final ColumnDescriptor descriptor; + + /** + * Whether to always return 128 bit decimals, regardless of its precision. If false, this will + * return 32, 64 or 128 bit decimals depending on the precision. + */ + protected final boolean useDecimal128; + + /** + * Whether to return dates/timestamps that were written with legacy hybrid (Julian + Gregorian) + * calendar as it is. If this is true, Comet will return them as it is, instead of rebasing them + * to the new Proleptic Gregorian calendar. If this is false, Comet will throw exceptions when + * seeing these dates/timestamps. + */ + protected final boolean useLegacyDateTimestamp; + + /** The size of one batch, gets updated by 'readBatch' */ + protected int batchSize; + + /** A pointer to the native implementation of ColumnReader. */ + protected long nativeHandle; + + public AbstractColumnReader( + DataType type, + ColumnDescriptor descriptor, + boolean useDecimal128, + boolean useLegacyDateTimestamp) { + this.type = type; + this.descriptor = descriptor; + this.useDecimal128 = useDecimal128; + this.useLegacyDateTimestamp = useLegacyDateTimestamp; + TypeUtil.checkParquetType(descriptor, type); + } + + public ColumnDescriptor getDescriptor() { + return descriptor; + } + + /** + * Set the batch size of this reader to be 'batchSize'. Also initializes the native column reader. + */ + public void setBatchSize(int batchSize) { + assert nativeHandle == 0 + : "Native column reader shouldn't be initialized before " + "'setBatchSize' is called"; + this.batchSize = batchSize; + initNative(); + } + + /** + * Reads a batch of 'total' new rows. + * + * @param total the total number of rows to read + */ + public abstract void readBatch(int total); + + /** Returns the {@link CometVector} read by this reader. */ + public abstract CometVector currentBatch(); + + @Override + public void close() { + if (nativeHandle != 0) { + LOG.debug("Closing the column reader"); + Native.closeColumnReader(nativeHandle); + nativeHandle = 0; + } + } + + protected void initNative() { + LOG.debug("initializing the native column reader"); + DataType readType = (boolean) CometConf.COMET_SCHEMA_EVOLUTION_ENABLED().get() ? type : null; + boolean useLegacyDateTimestampOrNTZ = + useLegacyDateTimestamp || type == TimestampNTZType$.MODULE$; + nativeHandle = + Utils.initColumnReader( + descriptor, readType, batchSize, useDecimal128, useLegacyDateTimestampOrNTZ); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/BatchReader.java b/common/src/main/java/org/apache/comet/parquet/BatchReader.java new file mode 100644 index 000000000..87302b372 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/BatchReader.java @@ -0,0 +1,620 @@ +/* + * 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.comet.parquet; + +import java.io.Closeable; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; + +import scala.Option; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.parquet.HadoopReadOptions; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.Preconditions; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.Type; +import org.apache.spark.TaskContext; +import org.apache.spark.TaskContext$; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.comet.parquet.CometParquetReadSupport; +import org.apache.spark.sql.execution.datasources.PartitionedFile; +import org.apache.spark.sql.execution.datasources.parquet.ParquetToSparkSchemaConverter; +import org.apache.spark.sql.execution.metric.SQLMetric; +import org.apache.spark.sql.types.*; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.apache.spark.util.AccumulatorV2; + +import org.apache.comet.CometConf; +import org.apache.comet.shims.ShimBatchReader; +import org.apache.comet.shims.ShimFileFormat; +import org.apache.comet.vector.CometVector; + +/** + * A vectorized Parquet reader that reads a Parquet file in a batched fashion. + * + *

Example of how to use this: + * + *

+ *   BatchReader reader = new BatchReader(parquetFile, batchSize);
+ *   try {
+ *     reader.init();
+ *     while (reader.readBatch()) {
+ *       ColumnarBatch batch = reader.currentBatch();
+ *       // consume the batch
+ *     }
+ *   } finally { // resources associated with the reader should be released
+ *     reader.close();
+ *   }
+ * 
+ */ +public class BatchReader extends RecordReader implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(FileReader.class); + + private Configuration conf; + private int capacity; + private boolean isCaseSensitive; + private boolean useFieldId; + private boolean ignoreMissingIds; + private StructType partitionSchema; + private InternalRow partitionValues; + private PartitionedFile file; + private final Map metrics; + + private long rowsRead; + private StructType sparkSchema; + private MessageType requestedSchema; + private CometVector[] vectors; + private AbstractColumnReader[] columnReaders; + private ColumnarBatch currentBatch; + private Future> prefetchTask; + private LinkedBlockingQueue> prefetchQueue; + private FileReader fileReader; + private boolean[] missingColumns; + private boolean isInitialized; + private ParquetMetadata footer; + + /** The total number of rows across all row groups of the input split. */ + private long totalRowCount; + + /** + * The total number of rows loaded so far, including all the rows from row groups that we've + * processed and the current row group. + */ + private long totalRowsLoaded; + + /** + * Whether the native scan should always return decimal represented by 128 bits, regardless of its + * precision. Normally, this should be true if native execution is enabled, since Arrow compute + * kernels doesn't support 32 and 64 bit decimals yet. + */ + private boolean useDecimal128; + + /** Whether to use the lazy materialization reader for reading columns. */ + private boolean useLazyMaterialization; + + /** + * Whether to return dates/timestamps that were written with legacy hybrid (Julian + Gregorian) + * calendar as it is. If this is true, Comet will return them as it is, instead of rebasing them + * to the new Proleptic Gregorian calendar. If this is false, Comet will throw exceptions when + * seeing these dates/timestamps. + */ + private boolean useLegacyDateTimestamp; + + /** The TaskContext object for executing this task. */ + private final TaskContext taskContext; + + // Only for testing + public BatchReader(String file, int capacity) { + this(file, capacity, null, null); + } + + // Only for testing + public BatchReader( + String file, int capacity, StructType partitionSchema, InternalRow partitionValues) { + this(new Configuration(), file, capacity, partitionSchema, partitionValues); + } + + // Only for testing + public BatchReader( + Configuration conf, + String file, + int capacity, + StructType partitionSchema, + InternalRow partitionValues) { + conf.set("spark.sql.parquet.binaryAsString", "false"); + conf.set("spark.sql.parquet.int96AsTimestamp", "false"); + conf.set("spark.sql.caseSensitive", "false"); + conf.set("spark.sql.parquet.inferTimestampNTZ.enabled", "true"); + conf.set("spark.sql.legacy.parquet.nanosAsLong", "false"); + + this.conf = conf; + this.capacity = capacity; + this.isCaseSensitive = false; + this.useFieldId = false; + this.ignoreMissingIds = false; + this.partitionSchema = partitionSchema; + this.partitionValues = partitionValues; + + this.file = ShimBatchReader.newPartitionedFile(partitionValues, file); + this.metrics = new HashMap<>(); + + this.taskContext = TaskContext$.MODULE$.get(); + } + + public BatchReader(AbstractColumnReader[] columnReaders) { + // Todo: set useDecimal128 and useLazyMaterialization + int numColumns = columnReaders.length; + this.columnReaders = new AbstractColumnReader[numColumns]; + vectors = new CometVector[numColumns]; + currentBatch = new ColumnarBatch(vectors); + // This constructor is used by Iceberg only. The columnReaders are + // initialized in Iceberg, so no need to call the init() + isInitialized = true; + this.taskContext = TaskContext$.MODULE$.get(); + this.metrics = new HashMap<>(); + } + + BatchReader( + Configuration conf, + PartitionedFile inputSplit, + ParquetMetadata footer, + int capacity, + StructType sparkSchema, + boolean isCaseSensitive, + boolean useFieldId, + boolean ignoreMissingIds, + boolean useLegacyDateTimestamp, + StructType partitionSchema, + InternalRow partitionValues, + Map metrics) { + this.conf = conf; + this.capacity = capacity; + this.sparkSchema = sparkSchema; + this.isCaseSensitive = isCaseSensitive; + this.useFieldId = useFieldId; + this.ignoreMissingIds = ignoreMissingIds; + this.useLegacyDateTimestamp = useLegacyDateTimestamp; + this.partitionSchema = partitionSchema; + this.partitionValues = partitionValues; + this.file = inputSplit; + this.footer = footer; + this.metrics = metrics; + this.taskContext = TaskContext$.MODULE$.get(); + } + + /** + * Initialize this reader. The reason we don't do it in the constructor is that we want to close + * any resource hold by this reader when error happens during the initialization. + */ + public void init() throws URISyntaxException, IOException { + useDecimal128 = + conf.getBoolean( + CometConf.COMET_USE_DECIMAL_128().key(), + (Boolean) CometConf.COMET_USE_DECIMAL_128().defaultValue().get()); + useLazyMaterialization = + conf.getBoolean( + CometConf.COMET_USE_LAZY_MATERIALIZATION().key(), + (Boolean) CometConf.COMET_USE_LAZY_MATERIALIZATION().defaultValue().get()); + + long start = file.start(); + long length = file.length(); + String filePath = file.filePath().toString(); + + ParquetReadOptions.Builder builder = HadoopReadOptions.builder(conf, new Path(filePath)); + + if (start >= 0 && length >= 0) { + builder = builder.withRange(start, start + length); + } + ParquetReadOptions readOptions = builder.build(); + + // TODO: enable off-heap buffer when they are ready + ReadOptions cometReadOptions = ReadOptions.builder(conf).build(); + + Path path = new Path(new URI(filePath)); + fileReader = + new FileReader( + CometInputFile.fromPath(path, conf), footer, readOptions, cometReadOptions, metrics); + requestedSchema = fileReader.getFileMetaData().getSchema(); + MessageType fileSchema = requestedSchema; + + if (sparkSchema == null) { + sparkSchema = new ParquetToSparkSchemaConverter(conf).convert(requestedSchema); + } else { + requestedSchema = + CometParquetReadSupport.clipParquetSchema( + requestedSchema, sparkSchema, isCaseSensitive, useFieldId, ignoreMissingIds); + if (requestedSchema.getColumns().size() != sparkSchema.size()) { + throw new IllegalArgumentException( + String.format( + "Spark schema has %d columns while " + "Parquet schema has %d columns", + sparkSchema.size(), requestedSchema.getColumns().size())); + } + } + + totalRowCount = fileReader.getRecordCount(); + List columns = requestedSchema.getColumns(); + int numColumns = columns.size(); + if (partitionSchema != null) numColumns += partitionSchema.size(); + columnReaders = new AbstractColumnReader[numColumns]; + + // Initialize missing columns and use null vectors for them + missingColumns = new boolean[columns.size()]; + List paths = requestedSchema.getPaths(); + StructField[] nonPartitionFields = sparkSchema.fields(); + for (int i = 0; i < requestedSchema.getFieldCount(); i++) { + Type t = requestedSchema.getFields().get(i); + Preconditions.checkState( + t.isPrimitive() && !t.isRepetition(Type.Repetition.REPEATED), + "Complex type is not supported"); + String[] colPath = paths.get(i); + if (nonPartitionFields[i].name().equals(ShimFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME())) { + // Values of ROW_INDEX_TEMPORARY_COLUMN_NAME column are always populated with + // generated row indexes, rather than read from the file. + // TODO(SPARK-40059): Allow users to include columns named + // FileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME in their schemas. + long[] rowIndices = fileReader.getRowIndices(); + columnReaders[i] = new RowIndexColumnReader(nonPartitionFields[i], capacity, rowIndices); + missingColumns[i] = true; + } else if (fileSchema.containsPath(colPath)) { + ColumnDescriptor fd = fileSchema.getColumnDescription(colPath); + if (!fd.equals(columns.get(i))) { + throw new UnsupportedOperationException("Schema evolution is not supported"); + } + missingColumns[i] = false; + } else { + if (columns.get(i).getMaxDefinitionLevel() == 0) { + throw new IOException( + "Required column '" + + Arrays.toString(colPath) + + "' is missing" + + " in data file " + + filePath); + } + ConstantColumnReader reader = + new ConstantColumnReader(nonPartitionFields[i], capacity, useDecimal128); + columnReaders[i] = reader; + missingColumns[i] = true; + } + } + + // Initialize constant readers for partition columns + if (partitionSchema != null) { + StructField[] partitionFields = partitionSchema.fields(); + for (int i = columns.size(); i < columnReaders.length; i++) { + int fieldIndex = i - columns.size(); + StructField field = partitionFields[fieldIndex]; + ConstantColumnReader reader = + new ConstantColumnReader(field, capacity, partitionValues, fieldIndex, useDecimal128); + columnReaders[i] = reader; + } + } + + vectors = new CometVector[numColumns]; + currentBatch = new ColumnarBatch(vectors); + fileReader.setRequestedSchema(requestedSchema.getColumns()); + + // For test purpose only + // If the last external accumulator is `NumRowGroupsAccumulator`, the row group number to read + // will be updated to the accumulator. So we can check if the row groups are filtered or not + // in test case. + // Note that this tries to get thread local TaskContext object, if this is called at other + // thread, it won't update the accumulator. + if (taskContext != null) { + Option> accu = taskContext.taskMetrics().externalAccums().lastOption(); + if (accu.isDefined() && accu.get().getClass().getSimpleName().equals("NumRowGroupsAcc")) { + @SuppressWarnings("unchecked") + AccumulatorV2 intAccum = (AccumulatorV2) accu.get(); + intAccum.add(fileReader.getRowGroups().size()); + } + } + + // Pre-fetching + boolean preFetchEnabled = + conf.getBoolean( + CometConf.COMET_SCAN_PREFETCH_ENABLED().key(), + (boolean) CometConf.COMET_SCAN_PREFETCH_ENABLED().defaultValue().get()); + + if (preFetchEnabled) { + LOG.info("Prefetch enabled for BatchReader."); + this.prefetchQueue = new LinkedBlockingQueue<>(); + } + + isInitialized = true; + synchronized (this) { + // if prefetch is enabled, `init()` is called in separate thread. When + // `BatchReader.nextBatch()` is called asynchronously, it is possibly that + // `init()` is not called or finished. We need to hold on `nextBatch` until + // initialization of `BatchReader` is done. Once we are close to finish + // initialization, we notify the waiting thread of `nextBatch` to continue. + notifyAll(); + } + } + + public void setSparkSchema(StructType schema) { + this.sparkSchema = schema; + } + + public AbstractColumnReader[] getColumnReaders() { + return columnReaders; + } + + @Override + public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) + throws IOException, InterruptedException { + // Do nothing. The initialization work is done in 'init' already. + } + + @Override + public boolean nextKeyValue() throws IOException { + return nextBatch(); + } + + @Override + public Void getCurrentKey() { + return null; + } + + @Override + public ColumnarBatch getCurrentValue() { + return currentBatch(); + } + + @Override + public float getProgress() { + return (float) rowsRead / totalRowCount; + } + + /** + * Returns the current columnar batch being read. + * + *

Note that this must be called AFTER {@link BatchReader#nextBatch()}. + */ + public ColumnarBatch currentBatch() { + return currentBatch; + } + + // Only for testing + public Future> getPrefetchTask() { + return this.prefetchTask; + } + + // Only for testing + public LinkedBlockingQueue> getPrefetchQueue() { + return this.prefetchQueue; + } + + /** + * Loads the next batch of rows. + * + * @return true if there are no more rows to read, false otherwise. + */ + public boolean nextBatch() throws IOException { + if (this.prefetchTask == null) { + Preconditions.checkState(isInitialized, "init() should be called first!"); + } else { + // If prefetch is enabled, this reader will be initialized asynchronously from a + // different thread. Wait until it is initialized + while (!isInitialized) { + synchronized (this) { + try { + // Wait until initialization of current `BatchReader` is finished (i.e., `init()`), + // is done. It is possibly that `init()` is done after entering this while loop, + // so a short timeout is given. + wait(100); + + // Checks if prefetch task is finished. If so, tries to get exception if any. + if (prefetchTask.isDone()) { + Option exception = prefetchTask.get(); + if (exception.isDefined()) { + throw exception.get(); + } + } + } catch (RuntimeException e) { + // Spark will check certain exception e.g. `SchemaColumnConvertNotSupportedException`. + throw e; + } catch (Throwable e) { + throw new IOException(e); + } + } + } + } + + if (rowsRead >= totalRowCount) return false; + boolean hasMore; + + try { + hasMore = loadNextRowGroupIfNecessary(); + } catch (RuntimeException e) { + // Spark will check certain exception e.g. `SchemaColumnConvertNotSupportedException`. + throw e; + } catch (Throwable e) { + throw new IOException(e); + } + + if (!hasMore) return false; + int batchSize = (int) Math.min(capacity, totalRowsLoaded - rowsRead); + + return nextBatch(batchSize); + } + + public boolean nextBatch(int batchSize) { + long totalDecodeTime = 0, totalLoadTime = 0; + for (int i = 0; i < columnReaders.length; i++) { + AbstractColumnReader reader = columnReaders[i]; + long startNs = System.nanoTime(); + reader.readBatch(batchSize); + totalDecodeTime += System.nanoTime() - startNs; + startNs = System.nanoTime(); + vectors[i] = reader.currentBatch(); + totalLoadTime += System.nanoTime() - startNs; + } + + SQLMetric decodeMetric = metrics.get("ParquetNativeDecodeTime"); + if (decodeMetric != null) { + decodeMetric.add(totalDecodeTime); + } + SQLMetric loadMetric = metrics.get("ParquetNativeLoadTime"); + if (loadMetric != null) { + loadMetric.add(totalLoadTime); + } + + currentBatch.setNumRows(batchSize); + rowsRead += batchSize; + return true; + } + + @Override + public void close() throws IOException { + if (columnReaders != null) { + for (AbstractColumnReader reader : columnReaders) { + if (reader != null) { + reader.close(); + } + } + } + if (fileReader != null) { + fileReader.close(); + fileReader = null; + } + } + + private boolean loadNextRowGroupIfNecessary() throws Throwable { + // More rows can be read from loaded row group. No need to load next one. + if (rowsRead != totalRowsLoaded) return true; + + SQLMetric rowGroupTimeMetric = metrics.get("ParquetLoadRowGroupTime"); + SQLMetric numRowGroupsMetric = metrics.get("ParquetRowGroups"); + long startNs = System.nanoTime(); + + PageReadStore rowGroupReader = null; + if (prefetchTask != null && prefetchQueue != null) { + // Wait for pre-fetch task to finish. + Pair rowGroupReaderPair = prefetchQueue.take(); + rowGroupReader = rowGroupReaderPair.getLeft(); + + // Update incremental byte read metric. Because this metric in Spark is maintained + // by thread local variable, we need to manually update it. + // TODO: We may expose metrics from `FileReader` and get from it directly. + long incBytesRead = rowGroupReaderPair.getRight(); + FileSystem.getAllStatistics().stream() + .forEach(statistic -> statistic.incrementBytesRead(incBytesRead)); + } else { + rowGroupReader = fileReader.readNextRowGroup(); + } + + if (rowGroupTimeMetric != null) { + rowGroupTimeMetric.add(System.nanoTime() - startNs); + } + if (rowGroupReader == null) { + return false; + } + if (numRowGroupsMetric != null) { + numRowGroupsMetric.add(1); + } + + List columns = requestedSchema.getColumns(); + for (int i = 0; i < columns.size(); i++) { + if (missingColumns[i]) continue; + if (columnReaders[i] != null) columnReaders[i].close(); + // TODO: handle tz, datetime & int96 rebase + // TODO: consider passing page reader via ctor - however we need to fix the shading issue + // from Iceberg side. + DataType dataType = sparkSchema.fields()[i].dataType(); + ColumnReader reader = + Utils.getColumnReader( + dataType, + columns.get(i), + capacity, + useDecimal128, + useLazyMaterialization, + useLegacyDateTimestamp); + reader.setPageReader(rowGroupReader.getPageReader(columns.get(i))); + columnReaders[i] = reader; + } + totalRowsLoaded += rowGroupReader.getRowCount(); + return true; + } + + // Submits a prefetch task for this reader. + public void submitPrefetchTask(ExecutorService threadPool) { + this.prefetchTask = threadPool.submit(new PrefetchTask()); + } + + // A task for prefetching parquet row groups. + private class PrefetchTask implements Callable> { + private long getBytesRead() { + return FileSystem.getAllStatistics().stream() + .mapToLong(s -> s.getThreadStatistics().getBytesRead()) + .sum(); + } + + @Override + public Option call() throws Exception { + // Gets the bytes read so far. + long baseline = getBytesRead(); + + try { + init(); + + while (true) { + PageReadStore rowGroupReader = fileReader.readNextRowGroup(); + + if (rowGroupReader == null) { + // Reaches the end of row groups. + return Option.empty(); + } else { + long incBytesRead = getBytesRead() - baseline; + + prefetchQueue.add(Pair.of(rowGroupReader, incBytesRead)); + } + } + } catch (Throwable e) { + // Returns exception thrown from the reader. The reader will re-throw it. + return Option.apply(e); + } finally { + if (fileReader != null) { + fileReader.closeStream(); + } + } + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/BloomFilterReader.java b/common/src/main/java/org/apache/comet/parquet/BloomFilterReader.java new file mode 100644 index 000000000..a23216c7f --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/BloomFilterReader.java @@ -0,0 +1,253 @@ +/* + * 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.comet.parquet; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.parquet.column.values.bloomfilter.BlockSplitBloomFilter; +import org.apache.parquet.column.values.bloomfilter.BloomFilter; +import org.apache.parquet.crypto.AesCipher; +import org.apache.parquet.crypto.InternalColumnDecryptionSetup; +import org.apache.parquet.crypto.InternalFileDecryptor; +import org.apache.parquet.crypto.ModuleCipherFactory; +import org.apache.parquet.crypto.ParquetCryptoRuntimeException; +import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.filter2.predicate.Operators; +import org.apache.parquet.filter2.predicate.UserDefinedPredicate; +import org.apache.parquet.format.BlockCipher; +import org.apache.parquet.format.BloomFilterHeader; +import org.apache.parquet.format.Util; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.parquet.io.SeekableInputStream; + +public class BloomFilterReader implements FilterPredicate.Visitor { + private static final Logger LOG = LoggerFactory.getLogger(BloomFilterReader.class); + private static final boolean BLOCK_MIGHT_MATCH = false; + private static final boolean BLOCK_CANNOT_MATCH = true; + + private final Map columns; + private final Map cache = new HashMap<>(); + private final InternalFileDecryptor fileDecryptor; + private final SeekableInputStream inputStream; + + BloomFilterReader( + BlockMetaData block, InternalFileDecryptor fileDecryptor, SeekableInputStream inputStream) { + this.columns = new HashMap<>(); + for (ColumnChunkMetaData column : block.getColumns()) { + columns.put(column.getPath(), column); + } + this.fileDecryptor = fileDecryptor; + this.inputStream = inputStream; + } + + @Override + public > Boolean visit(Operators.Eq eq) { + T value = eq.getValue(); + + if (value == null) { + // the bloom filter bitset contains only non-null values so isn't helpful. this + // could check the column stats, but the StatisticsFilter is responsible + return BLOCK_MIGHT_MATCH; + } + + Operators.Column filterColumn = eq.getColumn(); + ColumnChunkMetaData meta = columns.get(filterColumn.getColumnPath()); + if (meta == null) { + // the column isn't in this file so all values are null, but the value + // must be non-null because of the above check. + return BLOCK_CANNOT_MATCH; + } + + try { + BloomFilter bloomFilter = readBloomFilter(meta); + if (bloomFilter != null && !bloomFilter.findHash(bloomFilter.hash(value))) { + return BLOCK_CANNOT_MATCH; + } + } catch (RuntimeException e) { + LOG.warn(e.getMessage()); + return BLOCK_MIGHT_MATCH; + } + + return BLOCK_MIGHT_MATCH; + } + + @Override + public > Boolean visit(Operators.NotEq notEq) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public > Boolean visit(Operators.Lt lt) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public > Boolean visit(Operators.LtEq ltEq) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public > Boolean visit(Operators.Gt gt) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public > Boolean visit(Operators.GtEq gtEq) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public Boolean visit(Operators.And and) { + return and.getLeft().accept(this) || and.getRight().accept(this); + } + + @Override + public Boolean visit(Operators.Or or) { + return or.getLeft().accept(this) && or.getRight().accept(this); + } + + @Override + public Boolean visit(Operators.Not not) { + throw new IllegalArgumentException( + "This predicate " + + not + + " contains a not! Did you forget" + + " to run this predicate through LogicalInverseRewriter?"); + } + + @Override + public , U extends UserDefinedPredicate> Boolean visit( + Operators.UserDefined udp) { + return visit(udp, false); + } + + @Override + public , U extends UserDefinedPredicate> Boolean visit( + Operators.LogicalNotUserDefined udp) { + return visit(udp.getUserDefined(), true); + } + + private , U extends UserDefinedPredicate> Boolean visit( + Operators.UserDefined ud, boolean inverted) { + return BLOCK_MIGHT_MATCH; + } + + BloomFilter readBloomFilter(ColumnChunkMetaData meta) { + if (cache.containsKey(meta.getPath())) { + return cache.get(meta.getPath()); + } + try { + if (!cache.containsKey(meta.getPath())) { + BloomFilter bloomFilter = readBloomFilterInternal(meta); + if (bloomFilter == null) { + return null; + } + + cache.put(meta.getPath(), bloomFilter); + } + return cache.get(meta.getPath()); + } catch (IOException e) { + LOG.error("Failed to read Bloom filter data", e); + } + + return null; + } + + private BloomFilter readBloomFilterInternal(ColumnChunkMetaData meta) throws IOException { + long bloomFilterOffset = meta.getBloomFilterOffset(); + if (bloomFilterOffset < 0) { + return null; + } + + // Prepare to decrypt Bloom filter (for encrypted columns) + BlockCipher.Decryptor bloomFilterDecryptor = null; + byte[] bloomFilterHeaderAAD = null; + byte[] bloomFilterBitsetAAD = null; + if (null != fileDecryptor && !fileDecryptor.plaintextFile()) { + InternalColumnDecryptionSetup columnDecryptionSetup = + fileDecryptor.getColumnSetup(meta.getPath()); + if (columnDecryptionSetup.isEncrypted()) { + bloomFilterDecryptor = columnDecryptionSetup.getMetaDataDecryptor(); + bloomFilterHeaderAAD = + AesCipher.createModuleAAD( + fileDecryptor.getFileAAD(), + ModuleCipherFactory.ModuleType.BloomFilterHeader, + meta.getRowGroupOrdinal(), + columnDecryptionSetup.getOrdinal(), + -1); + bloomFilterBitsetAAD = + AesCipher.createModuleAAD( + fileDecryptor.getFileAAD(), + ModuleCipherFactory.ModuleType.BloomFilterBitset, + meta.getRowGroupOrdinal(), + columnDecryptionSetup.getOrdinal(), + -1); + } + } + + // Read Bloom filter data header. + inputStream.seek(bloomFilterOffset); + BloomFilterHeader bloomFilterHeader; + try { + bloomFilterHeader = + Util.readBloomFilterHeader(inputStream, bloomFilterDecryptor, bloomFilterHeaderAAD); + } catch (IOException e) { + LOG.warn("read no bloom filter"); + return null; + } + + int numBytes = bloomFilterHeader.getNumBytes(); + if (numBytes <= 0 || numBytes > BlockSplitBloomFilter.UPPER_BOUND_BYTES) { + LOG.warn("the read bloom filter size is wrong, size is {}", bloomFilterHeader.getNumBytes()); + return null; + } + + if (!bloomFilterHeader.getHash().isSetXXHASH() + || !bloomFilterHeader.getAlgorithm().isSetBLOCK() + || !bloomFilterHeader.getCompression().isSetUNCOMPRESSED()) { + LOG.warn( + "the read bloom filter is not supported yet, algorithm = {}, hash = {}, " + + "compression = {}", + bloomFilterHeader.getAlgorithm(), + bloomFilterHeader.getHash(), + bloomFilterHeader.getCompression()); + return null; + } + + byte[] bitset; + if (null == bloomFilterDecryptor) { + bitset = new byte[numBytes]; + inputStream.readFully(bitset); + } else { + bitset = bloomFilterDecryptor.decrypt(inputStream, bloomFilterBitsetAAD); + if (bitset.length != numBytes) { + throw new ParquetCryptoRuntimeException("Wrong length of decrypted bloom filter bitset"); + } + } + return new BlockSplitBloomFilter(bitset); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnIndexReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnIndexReader.java new file mode 100644 index 000000000..4e4f6ba0d --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/ColumnIndexReader.java @@ -0,0 +1,230 @@ +/* + * 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.comet.parquet; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.parquet.crypto.AesCipher; +import org.apache.parquet.crypto.InternalColumnDecryptionSetup; +import org.apache.parquet.crypto.InternalFileDecryptor; +import org.apache.parquet.crypto.ModuleCipherFactory; +import org.apache.parquet.format.BlockCipher; +import org.apache.parquet.format.Util; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.parquet.internal.column.columnindex.ColumnIndex; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.internal.filter2.columnindex.ColumnIndexStore; +import org.apache.parquet.internal.hadoop.metadata.IndexReference; +import org.apache.parquet.io.SeekableInputStream; + +class ColumnIndexReader implements ColumnIndexStore { + private static final Logger LOG = LoggerFactory.getLogger(ColumnIndexReader.class); + + // Used for columns are not in this parquet file + private static final IndexStore MISSING_INDEX_STORE = + new IndexStore() { + @Override + public ColumnIndex getColumnIndex() { + return null; + } + + @Override + public OffsetIndex getOffsetIndex() { + return null; + } + }; + + private static final ColumnIndexReader EMPTY = + new ColumnIndexReader(new BlockMetaData(), Collections.emptySet(), null, null) { + @Override + public ColumnIndex getColumnIndex(ColumnPath column) { + return null; + } + + @Override + public OffsetIndex getOffsetIndex(ColumnPath column) { + throw new MissingOffsetIndexException(column); + } + }; + + private final InternalFileDecryptor fileDecryptor; + private final SeekableInputStream inputStream; + private final Map store; + + /** + * Creates a column index store which lazily reads column/offset indexes for the columns in paths. + * Paths are the set of columns used for the projection. + */ + static ColumnIndexReader create( + BlockMetaData block, + Set paths, + InternalFileDecryptor fileDecryptor, + SeekableInputStream inputStream) { + try { + return new ColumnIndexReader(block, paths, fileDecryptor, inputStream); + } catch (MissingOffsetIndexException e) { + return EMPTY; + } + } + + private ColumnIndexReader( + BlockMetaData block, + Set paths, + InternalFileDecryptor fileDecryptor, + SeekableInputStream inputStream) { + this.fileDecryptor = fileDecryptor; + this.inputStream = inputStream; + Map store = new HashMap<>(); + for (ColumnChunkMetaData column : block.getColumns()) { + ColumnPath path = column.getPath(); + if (paths.contains(path)) { + store.put(path, new IndexStoreImpl(column)); + } + } + this.store = store; + } + + @Override + public ColumnIndex getColumnIndex(ColumnPath column) { + return store.getOrDefault(column, MISSING_INDEX_STORE).getColumnIndex(); + } + + @Override + public OffsetIndex getOffsetIndex(ColumnPath column) { + return store.getOrDefault(column, MISSING_INDEX_STORE).getOffsetIndex(); + } + + private interface IndexStore { + ColumnIndex getColumnIndex(); + + OffsetIndex getOffsetIndex(); + } + + private class IndexStoreImpl implements IndexStore { + private final ColumnChunkMetaData meta; + private ColumnIndex columnIndex; + private boolean columnIndexRead; + private final OffsetIndex offsetIndex; + + IndexStoreImpl(ColumnChunkMetaData meta) { + this.meta = meta; + OffsetIndex oi; + try { + oi = readOffsetIndex(meta); + } catch (IOException e) { + // If the I/O issue still stands it will fail the reading later; + // otherwise we fail the filtering only with a missing offset index. + LOG.warn("Unable to read offset index for column {}", meta.getPath(), e); + oi = null; + } + if (oi == null) { + throw new MissingOffsetIndexException(meta.getPath()); + } + offsetIndex = oi; + } + + @Override + public ColumnIndex getColumnIndex() { + if (!columnIndexRead) { + try { + columnIndex = readColumnIndex(meta); + } catch (IOException e) { + // If the I/O issue still stands it will fail the reading later; + // otherwise we fail the filtering only with a missing column index. + LOG.warn("Unable to read column index for column {}", meta.getPath(), e); + } + columnIndexRead = true; + } + return columnIndex; + } + + @Override + public OffsetIndex getOffsetIndex() { + return offsetIndex; + } + } + + // Visible for testing + ColumnIndex readColumnIndex(ColumnChunkMetaData column) throws IOException { + IndexReference ref = column.getColumnIndexReference(); + if (ref == null) { + return null; + } + inputStream.seek(ref.getOffset()); + + BlockCipher.Decryptor columnIndexDecryptor = null; + byte[] columnIndexAAD = null; + if (null != fileDecryptor && !fileDecryptor.plaintextFile()) { + InternalColumnDecryptionSetup columnDecryptionSetup = + fileDecryptor.getColumnSetup(column.getPath()); + if (columnDecryptionSetup.isEncrypted()) { + columnIndexDecryptor = columnDecryptionSetup.getMetaDataDecryptor(); + columnIndexAAD = + AesCipher.createModuleAAD( + fileDecryptor.getFileAAD(), + ModuleCipherFactory.ModuleType.ColumnIndex, + column.getRowGroupOrdinal(), + columnDecryptionSetup.getOrdinal(), + -1); + } + } + return ParquetMetadataConverter.fromParquetColumnIndex( + column.getPrimitiveType(), + Util.readColumnIndex(inputStream, columnIndexDecryptor, columnIndexAAD)); + } + + // Visible for testing + OffsetIndex readOffsetIndex(ColumnChunkMetaData column) throws IOException { + IndexReference ref = column.getOffsetIndexReference(); + if (ref == null) { + return null; + } + inputStream.seek(ref.getOffset()); + + BlockCipher.Decryptor offsetIndexDecryptor = null; + byte[] offsetIndexAAD = null; + if (null != fileDecryptor && !fileDecryptor.plaintextFile()) { + InternalColumnDecryptionSetup columnDecryptionSetup = + fileDecryptor.getColumnSetup(column.getPath()); + if (columnDecryptionSetup.isEncrypted()) { + offsetIndexDecryptor = columnDecryptionSetup.getMetaDataDecryptor(); + offsetIndexAAD = + AesCipher.createModuleAAD( + fileDecryptor.getFileAAD(), + ModuleCipherFactory.ModuleType.OffsetIndex, + column.getRowGroupOrdinal(), + columnDecryptionSetup.getOrdinal(), + -1); + } + } + return ParquetMetadataConverter.fromParquetOffsetIndex( + Util.readOffsetIndex(inputStream, offsetIndexDecryptor, offsetIndexAAD)); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnPageReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnPageReader.java new file mode 100644 index 000000000..744d12830 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/ColumnPageReader.java @@ -0,0 +1,252 @@ +/* + * 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.comet.parquet; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.List; +import java.util.Queue; + +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.column.page.DataPage; +import org.apache.parquet.column.page.DataPageV1; +import org.apache.parquet.column.page.DataPageV2; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.compression.CompressionCodecFactory; +import org.apache.parquet.crypto.AesCipher; +import org.apache.parquet.crypto.ModuleCipherFactory; +import org.apache.parquet.format.BlockCipher; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.io.ParquetDecodingException; + +public class ColumnPageReader implements PageReader { + private final CompressionCodecFactory.BytesInputDecompressor decompressor; + private final long valueCount; + private final Queue compressedPages; + private final DictionaryPage compressedDictionaryPage; + + private final OffsetIndex offsetIndex; + private final long rowCount; + private int pageIndex = 0; + + private final BlockCipher.Decryptor blockDecryptor; + private final byte[] dataPageAAD; + private final byte[] dictionaryPageAAD; + + ColumnPageReader( + CompressionCodecFactory.BytesInputDecompressor decompressor, + List compressedPages, + DictionaryPage compressedDictionaryPage, + OffsetIndex offsetIndex, + long rowCount, + BlockCipher.Decryptor blockDecryptor, + byte[] fileAAD, + int rowGroupOrdinal, + int columnOrdinal) { + this.decompressor = decompressor; + this.compressedPages = new ArrayDeque<>(compressedPages); + this.compressedDictionaryPage = compressedDictionaryPage; + long count = 0; + for (DataPage p : compressedPages) { + count += p.getValueCount(); + } + this.valueCount = count; + this.offsetIndex = offsetIndex; + this.rowCount = rowCount; + this.blockDecryptor = blockDecryptor; + + if (blockDecryptor != null) { + dataPageAAD = + AesCipher.createModuleAAD( + fileAAD, ModuleCipherFactory.ModuleType.DataPage, rowGroupOrdinal, columnOrdinal, 0); + dictionaryPageAAD = + AesCipher.createModuleAAD( + fileAAD, + ModuleCipherFactory.ModuleType.DictionaryPage, + rowGroupOrdinal, + columnOrdinal, + -1); + } else { + dataPageAAD = null; + dictionaryPageAAD = null; + } + } + + @Override + public long getTotalValueCount() { + return valueCount; + } + + /** Returns the total value count of the current page. */ + public int getPageValueCount() { + return compressedPages.element().getValueCount(); + } + + /** Skips the current page so it won't be returned by {@link #readPage()} */ + public void skipPage() { + compressedPages.poll(); + pageIndex++; + } + + @Override + public DataPage readPage() { + final DataPage compressedPage = compressedPages.poll(); + if (compressedPage == null) { + return null; + } + final int currentPageIndex = pageIndex++; + + if (null != blockDecryptor) { + AesCipher.quickUpdatePageAAD(dataPageAAD, getPageOrdinal(currentPageIndex)); + } + + return compressedPage.accept( + new DataPage.Visitor() { + @Override + public DataPage visit(DataPageV1 dataPageV1) { + try { + BytesInput bytes = dataPageV1.getBytes(); + if (null != blockDecryptor) { + bytes = BytesInput.from(blockDecryptor.decrypt(bytes.toByteArray(), dataPageAAD)); + } + BytesInput decompressed = + decompressor.decompress(bytes, dataPageV1.getUncompressedSize()); + + final DataPageV1 decompressedPage; + if (offsetIndex == null) { + decompressedPage = + new DataPageV1( + decompressed, + dataPageV1.getValueCount(), + dataPageV1.getUncompressedSize(), + dataPageV1.getStatistics(), + dataPageV1.getRlEncoding(), + dataPageV1.getDlEncoding(), + dataPageV1.getValueEncoding()); + } else { + long firstRowIndex = offsetIndex.getFirstRowIndex(currentPageIndex); + decompressedPage = + new DataPageV1( + decompressed, + dataPageV1.getValueCount(), + dataPageV1.getUncompressedSize(), + firstRowIndex, + Math.toIntExact( + offsetIndex.getLastRowIndex(currentPageIndex, rowCount) + - firstRowIndex + + 1), + dataPageV1.getStatistics(), + dataPageV1.getRlEncoding(), + dataPageV1.getDlEncoding(), + dataPageV1.getValueEncoding()); + } + if (dataPageV1.getCrc().isPresent()) { + decompressedPage.setCrc(dataPageV1.getCrc().getAsInt()); + } + return decompressedPage; + } catch (IOException e) { + throw new ParquetDecodingException("could not decompress page", e); + } + } + + @Override + public DataPage visit(DataPageV2 dataPageV2) { + if (!dataPageV2.isCompressed() && offsetIndex == null && null == blockDecryptor) { + return dataPageV2; + } + BytesInput pageBytes = dataPageV2.getData(); + + if (null != blockDecryptor) { + try { + pageBytes = + BytesInput.from(blockDecryptor.decrypt(pageBytes.toByteArray(), dataPageAAD)); + } catch (IOException e) { + throw new ParquetDecodingException( + "could not convert page ByteInput to byte array", e); + } + } + if (dataPageV2.isCompressed()) { + int uncompressedSize = + Math.toIntExact( + dataPageV2.getUncompressedSize() + - dataPageV2.getDefinitionLevels().size() + - dataPageV2.getRepetitionLevels().size()); + try { + pageBytes = decompressor.decompress(pageBytes, uncompressedSize); + } catch (IOException e) { + throw new ParquetDecodingException("could not decompress page", e); + } + } + + if (offsetIndex == null) { + return DataPageV2.uncompressed( + dataPageV2.getRowCount(), + dataPageV2.getNullCount(), + dataPageV2.getValueCount(), + dataPageV2.getRepetitionLevels(), + dataPageV2.getDefinitionLevels(), + dataPageV2.getDataEncoding(), + pageBytes, + dataPageV2.getStatistics()); + } else { + return DataPageV2.uncompressed( + dataPageV2.getRowCount(), + dataPageV2.getNullCount(), + dataPageV2.getValueCount(), + offsetIndex.getFirstRowIndex(currentPageIndex), + dataPageV2.getRepetitionLevels(), + dataPageV2.getDefinitionLevels(), + dataPageV2.getDataEncoding(), + pageBytes, + dataPageV2.getStatistics()); + } + } + }); + } + + @Override + public DictionaryPage readDictionaryPage() { + if (compressedDictionaryPage == null) { + return null; + } + try { + BytesInput bytes = compressedDictionaryPage.getBytes(); + if (null != blockDecryptor) { + bytes = BytesInput.from(blockDecryptor.decrypt(bytes.toByteArray(), dictionaryPageAAD)); + } + DictionaryPage decompressedPage = + new DictionaryPage( + decompressor.decompress(bytes, compressedDictionaryPage.getUncompressedSize()), + compressedDictionaryPage.getDictionarySize(), + compressedDictionaryPage.getEncoding()); + if (compressedDictionaryPage.getCrc().isPresent()) { + decompressedPage.setCrc(compressedDictionaryPage.getCrc().getAsInt()); + } + return decompressedPage; + } catch (IOException e) { + throw new ParquetDecodingException("Could not decompress dictionary page", e); + } + } + + private int getPageOrdinal(int currentPageIndex) { + return offsetIndex == null ? currentPageIndex : offsetIndex.getPageOrdinal(currentPageIndex); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java new file mode 100644 index 000000000..7e45f4f9a --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java @@ -0,0 +1,314 @@ +/* + * 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.comet.parquet; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.arrow.c.ArrowArray; +import org.apache.arrow.c.ArrowSchema; +import org.apache.arrow.c.CDataDictionaryProvider; +import org.apache.arrow.c.Data; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.dictionary.Dictionary; +import org.apache.arrow.vector.types.pojo.DictionaryEncoding; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.page.DataPage; +import org.apache.parquet.column.page.DataPageV1; +import org.apache.parquet.column.page.DataPageV2; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReader; +import org.apache.spark.sql.types.DataType; + +import org.apache.comet.CometConf; +import org.apache.comet.vector.CometDecodedVector; +import org.apache.comet.vector.CometDictionary; +import org.apache.comet.vector.CometDictionaryVector; +import org.apache.comet.vector.CometPlainVector; +import org.apache.comet.vector.CometVector; + +public class ColumnReader extends AbstractColumnReader { + protected static final Logger LOG = LoggerFactory.getLogger(ColumnReader.class); + protected static final BufferAllocator ALLOCATOR = new RootAllocator(); + + /** + * The current Comet vector holding all the values read by this column reader. Owned by this + * reader and MUST be closed after use. + */ + private CometDecodedVector currentVector; + + /** Dictionary values for this column. Only set if the column is using dictionary encoding. */ + protected CometDictionary dictionary; + + /** Reader for dictionary & data pages in the current column chunk. */ + protected PageReader pageReader; + + /** Whether the first data page has been loaded. */ + private boolean firstPageLoaded = false; + + /** + * The number of nulls in the current batch, used when we are skipping importing of Arrow vectors, + * in which case we'll simply update the null count of the existing vectors. + */ + int currentNumNulls; + + /** + * The number of values in the current batch, used when we are skipping importing of Arrow + * vectors, in which case we'll simply update the null count of the existing vectors. + */ + int currentNumValues; + + /** + * Whether the last loaded vector contains any null value. This is used to determine if we can + * skip vector reloading. If the flag is false, Arrow C API will skip to import the validity + * buffer, and therefore we cannot skip vector reloading. + */ + boolean hadNull; + + /** Dictionary provider for this column. */ + private final CDataDictionaryProvider dictionaryProvider = new CDataDictionaryProvider(); + + public ColumnReader( + DataType type, + ColumnDescriptor descriptor, + int batchSize, + boolean useDecimal128, + boolean useLegacyDateTimestamp) { + super(type, descriptor, useDecimal128, useLegacyDateTimestamp); + assert batchSize > 0 : "Batch size must be positive, found " + batchSize; + this.batchSize = batchSize; + initNative(); + } + + /** + * Set the page reader for a new column chunk to read. Expects to call `readBatch` after this. + * + * @param pageReader the page reader for the new column chunk + */ + public void setPageReader(PageReader pageReader) throws IOException { + this.pageReader = pageReader; + + DictionaryPage dictionaryPage = pageReader.readDictionaryPage(); + if (dictionaryPage != null) { + LOG.debug("dictionary page encoding = {}", dictionaryPage.getEncoding()); + Native.setDictionaryPage( + nativeHandle, + dictionaryPage.getDictionarySize(), + dictionaryPage.getBytes().toByteArray(), + dictionaryPage.getEncoding().ordinal()); + } + } + + @Override + public void readBatch(int total) { + LOG.debug("Start to batch of size = " + total); + + if (!firstPageLoaded) { + readPage(); + firstPageLoaded = true; + } + + // Now first reset the current columnar batch so that it can be used to fill in a new batch + // of values. Then, keep reading more data pages (via 'readBatch') until the current batch is + // full, or we have read 'total' number of values. + Native.resetBatch(nativeHandle); + + int left = total, nullsRead = 0; + while (left > 0) { + int[] array = Native.readBatch(nativeHandle, left); + int valuesRead = array[0]; + nullsRead += array[1]; + if (valuesRead < left) { + readPage(); + } + left -= valuesRead; + } + + this.currentNumValues = total; + this.currentNumNulls = nullsRead; + } + + /** Returns the {@link CometVector} read by this reader. */ + @Override + public CometVector currentBatch() { + return loadVector(); + } + + @Override + public void close() { + if (currentVector != null) { + currentVector.close(); + currentVector = null; + } + dictionaryProvider.close(); + super.close(); + } + + /** Returns a decoded {@link CometDecodedVector Comet vector}. */ + public CometDecodedVector loadVector() { + // Only re-use Comet vector iff: + // 1. if we're not using dictionary encoding, since with dictionary encoding, the native + // side may fallback to plain encoding and the underlying memory address for the vector + // will change as result. + // 2. if the column type is of fixed width, in other words, string/binary are not supported + // since the native side may resize the vector and therefore change memory address. + // 3. if the last loaded vector contains null values: if values of last vector are all not + // null, Arrow C data API will skip loading the native validity buffer, therefore we + // should not re-use the vector in that case. + // 4. if the last loaded vector doesn't contain any null value, but the current vector also + // are all not null, which means we can also re-use the loaded vector. + // 5. if the new number of value is the same or smaller + if ((hadNull || currentNumNulls == 0) + && currentVector != null + && dictionary == null + && currentVector.isFixedLength() + && currentVector.numValues() >= currentNumValues) { + currentVector.setNumNulls(currentNumNulls); + currentVector.setNumValues(currentNumValues); + return currentVector; + } + + LOG.debug("Reloading vector"); + + // Close the previous vector first to release struct memory allocated to import Arrow array & + // schema from native side, through the C data interface + if (currentVector != null) { + currentVector.close(); + } + + long[] addresses = Native.currentBatch(nativeHandle); + + try (ArrowArray array = ArrowArray.wrap(addresses[0]); + ArrowSchema schema = ArrowSchema.wrap(addresses[1])) { + FieldVector vector = Data.importVector(ALLOCATOR, array, schema, dictionaryProvider); + DictionaryEncoding dictionaryEncoding = vector.getField().getDictionary(); + + CometPlainVector cometVector = new CometPlainVector(vector, useDecimal128); + + // Update whether the current vector contains any null values. This is used in the following + // batch(s) to determine whether we can skip loading the native vector. + hadNull = cometVector.hasNull(); + + if (dictionaryEncoding == null) { + if (dictionary != null) { + // This means the column was using dictionary encoding but now has fall-back to plain + // encoding, on the native side. Setting 'dictionary' to null here, so we can use it as + // a condition to check if we can re-use vector later. + dictionary = null; + } + // Either the column is not dictionary encoded, or it was using dictionary encoding but + // a new data page has switched back to use plain encoding. For both cases we should + // return plain vector. + currentVector = cometVector; + return currentVector; + } else if (dictionary == null) { + // There is dictionary from native side but the Java side dictionary hasn't been + // initialized yet. + Dictionary arrowDictionary = dictionaryProvider.lookup(dictionaryEncoding.getId()); + CometPlainVector dictionaryVector = + new CometPlainVector(arrowDictionary.getVector(), useDecimal128); + dictionary = new CometDictionary(dictionaryVector); + } + + currentVector = + new CometDictionaryVector(cometVector, dictionary, dictionaryProvider, useDecimal128); + return currentVector; + } + } + + protected void readPage() { + DataPage page = pageReader.readPage(); + if (page == null) { + throw new RuntimeException("overreading: returned DataPage is null"); + } + ; + int pageValueCount = page.getValueCount(); + page.accept( + new DataPage.Visitor() { + @Override + public Void visit(DataPageV1 dataPageV1) { + LOG.debug("data page encoding = {}", dataPageV1.getValueEncoding()); + if (dataPageV1.getDlEncoding() != Encoding.RLE + && descriptor.getMaxDefinitionLevel() != 0) { + throw new UnsupportedOperationException( + "Unsupported encoding: " + dataPageV1.getDlEncoding()); + } + if (!isValidValueEncoding(dataPageV1.getValueEncoding())) { + throw new UnsupportedOperationException( + "Unsupported value encoding: " + dataPageV1.getValueEncoding()); + } + try { + boolean useDirectBuffer = + (Boolean) CometConf.COMET_PARQUET_ENABLE_DIRECT_BUFFER().get(); + if (useDirectBuffer) { + ByteBuffer buffer = dataPageV1.getBytes().toByteBuffer(); + Native.setPageBufferV1( + nativeHandle, pageValueCount, buffer, dataPageV1.getValueEncoding().ordinal()); + } else { + byte[] array = dataPageV1.getBytes().toByteArray(); + Native.setPageV1( + nativeHandle, pageValueCount, array, dataPageV1.getValueEncoding().ordinal()); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + return null; + } + + @Override + public Void visit(DataPageV2 dataPageV2) { + if (!isValidValueEncoding(dataPageV2.getDataEncoding())) { + throw new UnsupportedOperationException( + "Unsupported encoding: " + dataPageV2.getDataEncoding()); + } + try { + Native.setPageV2( + nativeHandle, + pageValueCount, + dataPageV2.getDefinitionLevels().toByteArray(), + dataPageV2.getRepetitionLevels().toByteArray(), + dataPageV2.getData().toByteArray(), + dataPageV2.getDataEncoding().ordinal()); + } catch (IOException e) { + throw new RuntimeException(e); + } + return null; + } + }); + } + + @SuppressWarnings("deprecation") + private boolean isValidValueEncoding(Encoding encoding) { + switch (encoding) { + case PLAIN: + case RLE_DICTIONARY: + case PLAIN_DICTIONARY: + return true; + default: + return false; + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/CometInputFile.java b/common/src/main/java/org/apache/comet/parquet/CometInputFile.java new file mode 100644 index 000000000..eb54d1a72 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/CometInputFile.java @@ -0,0 +1,157 @@ +/* + * 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.comet.parquet; + +import java.io.IOException; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FutureDataInputStreamBuilder; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.util.VersionInfo; +import org.apache.parquet.hadoop.util.HadoopStreams; +import org.apache.parquet.io.InputFile; +import org.apache.parquet.io.SeekableInputStream; + +/** + * A Parquet {@link InputFile} implementation that's similar to {@link + * org.apache.parquet.hadoop.util.HadoopInputFile}, but with optimizations introduced in Hadoop 3.x, + * for S3 specifically. + */ +public class CometInputFile implements InputFile { + private static final String MAJOR_MINOR_REGEX = "^(\\d+)\\.(\\d+)(\\..*)?$"; + private static final Pattern VERSION_MATCHER = Pattern.compile(MAJOR_MINOR_REGEX); + + private final FileSystem fs; + private final FileStatus stat; + private final Configuration conf; + + public static CometInputFile fromPath(Path path, Configuration conf) throws IOException { + FileSystem fs = path.getFileSystem(conf); + return new CometInputFile(fs, fs.getFileStatus(path), conf); + } + + private CometInputFile(FileSystem fs, FileStatus stat, Configuration conf) { + this.fs = fs; + this.stat = stat; + this.conf = conf; + } + + @Override + public long getLength() { + return stat.getLen(); + } + + public Configuration getConf() { + return this.conf; + } + + public FileSystem getFileSystem() { + return this.fs; + } + + public Path getPath() { + return stat.getPath(); + } + + @Override + public SeekableInputStream newStream() throws IOException { + FSDataInputStream stream; + try { + if (isAtLeastHadoop33()) { + // If Hadoop version is >= 3.3.x, we'll use the 'openFile' API which can save a + // HEAD request from cloud storages like S3 + FutureDataInputStreamBuilder inputStreamBuilder = + fs.openFile(stat.getPath()).withFileStatus(stat); + + if (stat.getPath().toString().startsWith("s3a")) { + // Switch to random S3 input policy so that we don't do sequential read on the entire + // S3 object. By default, the policy is normal which does sequential read until a back + // seek happens, which in our case will never happen. + inputStreamBuilder = + inputStreamBuilder.opt("fs.s3a.experimental.input.fadvise", "random"); + } + stream = inputStreamBuilder.build().get(); + } else { + stream = fs.open(stat.getPath()); + } + } catch (Exception e) { + throw new IOException("Error when opening file " + stat.getPath(), e); + } + return HadoopStreams.wrap(stream); + } + + public SeekableInputStream newStream(long offset, long length) throws IOException { + try { + FSDataInputStream stream; + if (isAtLeastHadoop33()) { + FutureDataInputStreamBuilder inputStreamBuilder = + fs.openFile(stat.getPath()).withFileStatus(stat); + + if (stat.getPath().toString().startsWith("s3a")) { + // Switch to random S3 input policy so that we don't do sequential read on the entire + // S3 object. By default, the policy is normal which does sequential read until a back + // seek happens, which in our case will never happen. + // + // Also set read ahead length equal to the column chunk length so we don't have to open + // multiple S3 http connections. + inputStreamBuilder = + inputStreamBuilder + .opt("fs.s3a.experimental.input.fadvise", "random") + .opt("fs.s3a.readahead.range", Long.toString(length)); + } + + stream = inputStreamBuilder.build().get(); + } else { + stream = fs.open(stat.getPath()); + } + return HadoopStreams.wrap(stream); + } catch (Exception e) { + throw new IOException( + "Error when opening file " + stat.getPath() + ", offset=" + offset + ", length=" + length, + e); + } + } + + @Override + public String toString() { + return stat.getPath().toString(); + } + + private static boolean isAtLeastHadoop33() { + String version = VersionInfo.getVersion(); + return CometInputFile.isAtLeastHadoop33(version); + } + + static boolean isAtLeastHadoop33(String version) { + Matcher matcher = VERSION_MATCHER.matcher(version); + if (matcher.matches()) { + if (matcher.group(1).equals("3")) { + int minorVersion = Integer.parseInt(matcher.group(2)); + return minorVersion >= 3; + } + } + return false; + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java new file mode 100644 index 000000000..8de2376f9 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java @@ -0,0 +1,126 @@ +/* + * 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.comet.parquet; + +import java.math.BigInteger; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.*; +import org.apache.spark.unsafe.types.UTF8String; + +import org.apache.comet.shims.ShimResolveDefaultColumns; + +/** + * A column reader that always return constant vectors. Used for reading partition columns, for + * instance. + */ +public class ConstantColumnReader extends MetadataColumnReader { + /** Whether all the values in this constant column are nulls */ + private boolean isNull; + + /** The constant value in the format of Object that are used to initialize this column reader. */ + private Object value; + + public ConstantColumnReader(StructField field, int batchSize, boolean useDecimal128) { + this(field.dataType(), TypeUtil.convertToParquet(field), batchSize, useDecimal128); + this.value = ShimResolveDefaultColumns.getExistenceDefaultValue(field); + init(value); + } + + public ConstantColumnReader( + StructField field, int batchSize, InternalRow values, int index, boolean useDecimal128) { + this(field.dataType(), TypeUtil.convertToParquet(field), batchSize, useDecimal128); + init(values, index); + } + + public ConstantColumnReader( + DataType type, ColumnDescriptor descriptor, Object value, boolean useDecimal128) { + super(type, descriptor, useDecimal128); + this.value = value; + } + + ConstantColumnReader( + DataType type, ColumnDescriptor descriptor, int batchSize, boolean useDecimal128) { + super(type, descriptor, useDecimal128); + this.batchSize = batchSize; + initNative(); + } + + @Override + public void setBatchSize(int batchSize) { + super.setBatchSize(batchSize); + init(value); + } + + @Override + public void readBatch(int total) { + super.readBatch(total); + if (isNull) setNumNulls(total); + } + + private void init(InternalRow values, int index) { + Object value = values.get(index, type); + init(value); + } + + private void init(Object value) { + if (value == null) { + Native.setNull(nativeHandle); + isNull = true; + } else if (type == DataTypes.BooleanType) { + Native.setBoolean(nativeHandle, (boolean) value); + } else if (type == DataTypes.ByteType) { + Native.setByte(nativeHandle, (byte) value); + } else if (type == DataTypes.ShortType) { + Native.setShort(nativeHandle, (short) value); + } else if (type == DataTypes.IntegerType) { + Native.setInt(nativeHandle, (int) value); + } else if (type == DataTypes.LongType) { + Native.setLong(nativeHandle, (long) value); + } else if (type == DataTypes.FloatType) { + Native.setFloat(nativeHandle, (float) value); + } else if (type == DataTypes.DoubleType) { + Native.setDouble(nativeHandle, (double) value); + } else if (type == DataTypes.BinaryType) { + Native.setBinary(nativeHandle, (byte[]) value); + } else if (type == DataTypes.StringType) { + Native.setBinary(nativeHandle, ((UTF8String) value).getBytes()); + } else if (type == DataTypes.DateType) { + Native.setInt(nativeHandle, (int) value); + } else if (type == DataTypes.TimestampType || type == TimestampNTZType$.MODULE$) { + Native.setLong(nativeHandle, (long) value); + } else if (type instanceof DecimalType) { + DecimalType dt = (DecimalType) type; + Decimal d = (Decimal) value; + if (!useDecimal128 && dt.precision() <= Decimal.MAX_INT_DIGITS()) { + Native.setInt(nativeHandle, ((int) d.toUnscaledLong())); + } else if (!useDecimal128 && dt.precision() <= Decimal.MAX_LONG_DIGITS()) { + Native.setLong(nativeHandle, d.toUnscaledLong()); + } else { + final BigInteger integer = d.toJavaBigDecimal().unscaledValue(); + byte[] bytes = integer.toByteArray(); + Native.setDecimal(nativeHandle, bytes); + } + } else { + throw new UnsupportedOperationException("Unsupported Spark type: " + type); + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/DictionaryPageReader.java b/common/src/main/java/org/apache/comet/parquet/DictionaryPageReader.java new file mode 100644 index 000000000..b4b5a8fc9 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/DictionaryPageReader.java @@ -0,0 +1,190 @@ +/* + * 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.comet.parquet; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.DictionaryPageReadStore; +import org.apache.parquet.compression.CompressionCodecFactory; +import org.apache.parquet.crypto.AesCipher; +import org.apache.parquet.crypto.InternalColumnDecryptionSetup; +import org.apache.parquet.crypto.InternalFileDecryptor; +import org.apache.parquet.crypto.ModuleCipherFactory; +import org.apache.parquet.format.BlockCipher; +import org.apache.parquet.format.DictionaryPageHeader; +import org.apache.parquet.format.PageHeader; +import org.apache.parquet.format.Util; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.io.ParquetDecodingException; +import org.apache.parquet.io.SeekableInputStream; + +public class DictionaryPageReader implements DictionaryPageReadStore { + private final Map> cache; + private final InternalFileDecryptor fileDecryptor; + private final SeekableInputStream inputStream; + private final ParquetReadOptions options; + private final Map columns; + + DictionaryPageReader( + BlockMetaData block, + InternalFileDecryptor fileDecryptor, + SeekableInputStream inputStream, + ParquetReadOptions options) { + this.columns = new HashMap<>(); + this.cache = new ConcurrentHashMap<>(); + this.fileDecryptor = fileDecryptor; + this.inputStream = inputStream; + this.options = options; + + for (ColumnChunkMetaData column : block.getColumns()) { + columns.put(column.getPath().toDotString(), column); + } + } + + @Override + public DictionaryPage readDictionaryPage(ColumnDescriptor descriptor) { + String dotPath = String.join(".", descriptor.getPath()); + ColumnChunkMetaData column = columns.get(dotPath); + + if (column == null) { + throw new ParquetDecodingException("Failed to load dictionary, unknown column: " + dotPath); + } + + return cache + .computeIfAbsent( + dotPath, + key -> { + try { + final DictionaryPage dict = + column.hasDictionaryPage() ? readDictionary(column) : null; + + // Copy the dictionary to ensure it can be reused if it is returned + // more than once. This can happen when a DictionaryFilter has two or + // more predicates for the same column. Cache misses as well. + return (dict != null) ? Optional.of(reusableCopy(dict)) : Optional.empty(); + } catch (IOException e) { + throw new ParquetDecodingException("Failed to read dictionary", e); + } + }) + .orElse(null); + } + + DictionaryPage readDictionary(ColumnChunkMetaData meta) throws IOException { + if (!meta.hasDictionaryPage()) { + return null; + } + + if (inputStream.getPos() != meta.getStartingPos()) { + inputStream.seek(meta.getStartingPos()); + } + + boolean encryptedColumn = false; + InternalColumnDecryptionSetup columnDecryptionSetup = null; + byte[] dictionaryPageAAD = null; + BlockCipher.Decryptor pageDecryptor = null; + if (null != fileDecryptor && !fileDecryptor.plaintextFile()) { + columnDecryptionSetup = fileDecryptor.getColumnSetup(meta.getPath()); + if (columnDecryptionSetup.isEncrypted()) { + encryptedColumn = true; + } + } + + PageHeader pageHeader; + if (!encryptedColumn) { + pageHeader = Util.readPageHeader(inputStream); + } else { + byte[] dictionaryPageHeaderAAD = + AesCipher.createModuleAAD( + fileDecryptor.getFileAAD(), + ModuleCipherFactory.ModuleType.DictionaryPageHeader, + meta.getRowGroupOrdinal(), + columnDecryptionSetup.getOrdinal(), + -1); + pageHeader = + Util.readPageHeader( + inputStream, columnDecryptionSetup.getMetaDataDecryptor(), dictionaryPageHeaderAAD); + dictionaryPageAAD = + AesCipher.createModuleAAD( + fileDecryptor.getFileAAD(), + ModuleCipherFactory.ModuleType.DictionaryPage, + meta.getRowGroupOrdinal(), + columnDecryptionSetup.getOrdinal(), + -1); + pageDecryptor = columnDecryptionSetup.getDataDecryptor(); + } + + if (!pageHeader.isSetDictionary_page_header()) { + return null; + } + + DictionaryPage compressedPage = + readCompressedDictionary(pageHeader, inputStream, pageDecryptor, dictionaryPageAAD); + CompressionCodecFactory.BytesInputDecompressor decompressor = + options.getCodecFactory().getDecompressor(meta.getCodec()); + + return new DictionaryPage( + decompressor.decompress(compressedPage.getBytes(), compressedPage.getUncompressedSize()), + compressedPage.getDictionarySize(), + compressedPage.getEncoding()); + } + + private DictionaryPage readCompressedDictionary( + PageHeader pageHeader, + SeekableInputStream fin, + BlockCipher.Decryptor pageDecryptor, + byte[] dictionaryPageAAD) + throws IOException { + DictionaryPageHeader dictHeader = pageHeader.getDictionary_page_header(); + + int uncompressedPageSize = pageHeader.getUncompressed_page_size(); + int compressedPageSize = pageHeader.getCompressed_page_size(); + + byte[] dictPageBytes = new byte[compressedPageSize]; + fin.readFully(dictPageBytes); + + BytesInput bin = BytesInput.from(dictPageBytes); + + if (null != pageDecryptor) { + bin = BytesInput.from(pageDecryptor.decrypt(bin.toByteArray(), dictionaryPageAAD)); + } + + return new DictionaryPage( + bin, + uncompressedPageSize, + dictHeader.getNum_values(), + org.apache.parquet.column.Encoding.valueOf(dictHeader.getEncoding().name())); + } + + private static DictionaryPage reusableCopy(DictionaryPage dict) throws IOException { + return new DictionaryPage( + BytesInput.from(dict.getBytes().toByteArray()), + dict.getDictionarySize(), + dict.getEncoding()); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/FileReader.java b/common/src/main/java/org/apache/comet/parquet/FileReader.java new file mode 100644 index 000000000..eddaf3f1a --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/FileReader.java @@ -0,0 +1,1151 @@ +/* + * 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.comet.parquet; + +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.lang.reflect.Method; +import java.net.URI; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.zip.CRC32; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.Preconditions; +import org.apache.parquet.bytes.ByteBufferInputStream; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.bytes.BytesUtils; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.DataPage; +import org.apache.parquet.column.page.DataPageV1; +import org.apache.parquet.column.page.DataPageV2; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.compression.CompressionCodecFactory; +import org.apache.parquet.crypto.AesCipher; +import org.apache.parquet.crypto.FileDecryptionProperties; +import org.apache.parquet.crypto.InternalColumnDecryptionSetup; +import org.apache.parquet.crypto.InternalFileDecryptor; +import org.apache.parquet.crypto.ModuleCipherFactory; +import org.apache.parquet.crypto.ParquetCryptoRuntimeException; +import org.apache.parquet.filter2.compat.FilterCompat; +import org.apache.parquet.format.BlockCipher; +import org.apache.parquet.format.DataPageHeader; +import org.apache.parquet.format.DataPageHeaderV2; +import org.apache.parquet.format.DictionaryPageHeader; +import org.apache.parquet.format.FileCryptoMetaData; +import org.apache.parquet.format.PageHeader; +import org.apache.parquet.format.Util; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.parquet.hadoop.metadata.FileMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.hadoop.util.counters.BenchmarkCounter; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.internal.filter2.columnindex.ColumnIndexFilter; +import org.apache.parquet.internal.filter2.columnindex.ColumnIndexStore; +import org.apache.parquet.internal.filter2.columnindex.RowRanges; +import org.apache.parquet.io.InputFile; +import org.apache.parquet.io.ParquetDecodingException; +import org.apache.parquet.io.SeekableInputStream; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.spark.sql.execution.metric.SQLMetric; + +import static org.apache.parquet.hadoop.ParquetFileWriter.EFMAGIC; +import static org.apache.parquet.hadoop.ParquetFileWriter.MAGIC; + +import static org.apache.comet.parquet.RowGroupFilter.FilterLevel.BLOOMFILTER; +import static org.apache.comet.parquet.RowGroupFilter.FilterLevel.DICTIONARY; +import static org.apache.comet.parquet.RowGroupFilter.FilterLevel.STATISTICS; + +/** + * A Parquet file reader. Mostly followed {@code ParquetFileReader} in {@code parquet-mr}, but with + * customizations & optimizations for Comet. + */ +public class FileReader implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(FileReader.class); + + private final ParquetMetadataConverter converter; + protected final SeekableInputStream f; + private final InputFile file; + private final Map metrics; + private final Map paths = new HashMap<>(); + private final FileMetaData fileMetaData; // may be null + private final List blocks; + private final List blockIndexStores; + private final List blockRowRanges; + private final CRC32 crc; + private final ParquetMetadata footer; + + /** + * Read configurations come from two options: - options: these are options defined & specified + * from 'parquet-mr' library - cometOptions: these are Comet-specific options, for the features + * introduced in Comet's Parquet implementation + */ + private final ParquetReadOptions options; + + private final ReadOptions cometOptions; + + private int currentBlock = 0; + private RowGroupReader currentRowGroup = null; + private InternalFileDecryptor fileDecryptor; + + public FileReader(InputFile file, ParquetReadOptions options, ReadOptions cometOptions) + throws IOException { + this(file, null, options, cometOptions, null); + } + + public FileReader( + InputFile file, + ParquetReadOptions options, + ReadOptions cometOptions, + Map metrics) + throws IOException { + this(file, null, options, cometOptions, metrics); + } + + public FileReader( + InputFile file, + ParquetMetadata footer, + ParquetReadOptions options, + ReadOptions cometOptions, + Map metrics) + throws IOException { + this.converter = new ParquetMetadataConverter(options); + this.file = file; + this.f = file.newStream(); + this.options = options; + this.cometOptions = cometOptions; + this.metrics = metrics; + if (footer == null) { + try { + footer = readFooter(file, options, f, converter); + } catch (Exception e) { + // In case that reading footer throws an exception in the constructor, the new stream + // should be closed. Otherwise, there's no way to close this outside. + f.close(); + throw e; + } + } + this.footer = footer; + this.fileMetaData = footer.getFileMetaData(); + this.fileDecryptor = fileMetaData.getFileDecryptor(); // must be called before filterRowGroups! + if (null != fileDecryptor && fileDecryptor.plaintextFile()) { + this.fileDecryptor = null; // Plaintext file. No need in decryptor + } + + this.blocks = filterRowGroups(footer.getBlocks()); + this.blockIndexStores = listWithNulls(this.blocks.size()); + this.blockRowRanges = listWithNulls(this.blocks.size()); + for (ColumnDescriptor col : footer.getFileMetaData().getSchema().getColumns()) { + paths.put(ColumnPath.get(col.getPath()), col); + } + this.crc = options.usePageChecksumVerification() ? new CRC32() : null; + } + + /** Returns the footer of the Parquet file being read. */ + public ParquetMetadata getFooter() { + return this.footer; + } + + /** Returns the metadata of the Parquet file being read. */ + public FileMetaData getFileMetaData() { + return this.fileMetaData; + } + + /** Returns the input stream of the Parquet file being read. */ + public SeekableInputStream getInputStream() { + return this.f; + } + + /** Returns the Parquet options for reading the file. */ + public ParquetReadOptions getOptions() { + return this.options; + } + + /** Returns all the row groups of this reader (after applying row group filtering). */ + public List getRowGroups() { + return blocks; + } + + /** Sets the projected columns to be read later via {@link #readNextRowGroup()} */ + public void setRequestedSchema(List projection) { + paths.clear(); + for (ColumnDescriptor col : projection) { + paths.put(ColumnPath.get(col.getPath()), col); + } + } + + /** + * Gets the total number of records across all row groups (after applying row group filtering). + */ + public long getRecordCount() { + long total = 0; + for (BlockMetaData block : blocks) { + total += block.getRowCount(); + } + return total; + } + + /** + * Gets the total number of records across all row groups (after applying both row group filtering + * and page-level column index filtering). + */ + public long getFilteredRecordCount() { + if (!options.useColumnIndexFilter() + || !FilterCompat.isFilteringRequired(options.getRecordFilter())) { + return getRecordCount(); + } + long total = 0; + for (int i = 0, n = blocks.size(); i < n; ++i) { + total += getRowRanges(i).rowCount(); + } + return total; + } + + /** Skips the next row group. Returns false if there's no row group to skip. Otherwise, true. */ + public boolean skipNextRowGroup() { + return advanceToNextBlock(); + } + + /** + * Returns the next row group to read (after applying row group filtering), or null if there's no + * more row group. + */ + public PageReadStore readNextRowGroup() throws IOException { + if (currentBlock == blocks.size()) { + return null; + } + BlockMetaData block = blocks.get(currentBlock); + if (block.getRowCount() == 0) { + throw new RuntimeException("Illegal row group of 0 rows"); + } + this.currentRowGroup = new RowGroupReader(block.getRowCount()); + // prepare the list of consecutive parts to read them in one scan + List allParts = new ArrayList<>(); + ConsecutivePartList currentParts = null; + for (ColumnChunkMetaData mc : block.getColumns()) { + ColumnPath pathKey = mc.getPath(); + ColumnDescriptor columnDescriptor = paths.get(pathKey); + if (columnDescriptor != null) { + BenchmarkCounter.incrementTotalBytes(mc.getTotalSize()); + long startingPos = mc.getStartingPos(); + boolean mergeRanges = cometOptions.isIOMergeRangesEnabled(); + int mergeRangeDelta = cometOptions.getIOMergeRangesDelta(); + + // start a new list if - + // it is the first part or + // the part is consecutive or + // the part is not consecutive but within the merge range + if (currentParts == null + || (!mergeRanges && currentParts.endPos() != startingPos) + || (mergeRanges && startingPos - currentParts.endPos() > mergeRangeDelta)) { + currentParts = new ConsecutivePartList(startingPos); + allParts.add(currentParts); + } + // if we are in a consecutive part list and there is a gap in between the parts, + // we treat the gap as a skippable chunk + long delta = startingPos - currentParts.endPos(); + if (mergeRanges && delta > 0 && delta <= mergeRangeDelta) { + // add a chunk that will be skipped because it has no column descriptor + currentParts.addChunk(new ChunkDescriptor(null, null, startingPos, delta)); + } + currentParts.addChunk( + new ChunkDescriptor(columnDescriptor, mc, startingPos, mc.getTotalSize())); + } + } + // actually read all the chunks + return readChunks(block, allParts, new ChunkListBuilder()); + } + + /** + * Returns the next row group to read (after applying both row group filtering and page level + * column index filtering), or null if there's no more row group. + */ + public PageReadStore readNextFilteredRowGroup() throws IOException { + if (currentBlock == blocks.size()) { + return null; + } + if (!options.useColumnIndexFilter() + || !FilterCompat.isFilteringRequired(options.getRecordFilter())) { + return readNextRowGroup(); + } + BlockMetaData block = blocks.get(currentBlock); + if (block.getRowCount() == 0) { + throw new RuntimeException("Illegal row group of 0 rows"); + } + ColumnIndexStore ciStore = getColumnIndexReader(currentBlock); + RowRanges rowRanges = getRowRanges(currentBlock); + long rowCount = rowRanges.rowCount(); + if (rowCount == 0) { + // There are no matching rows -> skipping this row-group + advanceToNextBlock(); + return readNextFilteredRowGroup(); + } + if (rowCount == block.getRowCount()) { + // All rows are matching -> fall back to the non-filtering path + return readNextRowGroup(); + } + + this.currentRowGroup = new RowGroupReader(rowRanges); + // prepare the list of consecutive parts to read them in one scan + ChunkListBuilder builder = new ChunkListBuilder(); + List allParts = new ArrayList<>(); + ConsecutivePartList currentParts = null; + for (ColumnChunkMetaData mc : block.getColumns()) { + ColumnPath pathKey = mc.getPath(); + ColumnDescriptor columnDescriptor = paths.get(pathKey); + if (columnDescriptor != null) { + OffsetIndex offsetIndex = ciStore.getOffsetIndex(mc.getPath()); + IndexFilter indexFilter = new IndexFilter(rowRanges, offsetIndex, block.getRowCount()); + OffsetIndex filteredOffsetIndex = indexFilter.filterOffsetIndex(); + for (IndexFilter.OffsetRange range : + indexFilter.calculateOffsetRanges(filteredOffsetIndex, mc)) { + BenchmarkCounter.incrementTotalBytes(range.length); + long startingPos = range.offset; + // first part or not consecutive => new list + if (currentParts == null || currentParts.endPos() != startingPos) { + currentParts = new ConsecutivePartList(startingPos); + allParts.add(currentParts); + } + ChunkDescriptor chunkDescriptor = + new ChunkDescriptor(columnDescriptor, mc, startingPos, range.length); + currentParts.addChunk(chunkDescriptor); + builder.setOffsetIndex(chunkDescriptor, filteredOffsetIndex); + } + } + } + // actually read all the chunks + return readChunks(block, allParts, builder); + } + + // Visible for testing + ColumnIndexReader getColumnIndexReader(int blockIndex) { + ColumnIndexReader ciStore = blockIndexStores.get(blockIndex); + if (ciStore == null) { + ciStore = ColumnIndexReader.create(blocks.get(blockIndex), paths.keySet(), fileDecryptor, f); + blockIndexStores.set(blockIndex, ciStore); + } + return ciStore; + } + + private PageReadStore readChunks( + BlockMetaData block, List allParts, ChunkListBuilder builder) + throws IOException { + for (ConsecutivePartList consecutiveChunks : allParts) { + if (shouldReadParallel()) { + consecutiveChunks.readAllParallel(builder); + } else { + consecutiveChunks.readAll(f, builder); + } + } + for (Chunk chunk : builder.build()) { + readChunkPages(chunk, block); + } + + advanceToNextBlock(); + + return currentRowGroup; + } + + private boolean shouldReadParallel() { + if (file instanceof CometInputFile) { + URI uri = ((CometInputFile) file).getPath().toUri(); + return shouldReadParallel(cometOptions, uri.getScheme()); + } + + return false; + } + + static boolean shouldReadParallel(ReadOptions options, String scheme) { + return options.isParallelIOEnabled() && shouldReadParallelForScheme(scheme); + } + + private static boolean shouldReadParallelForScheme(String scheme) { + if (scheme == null) { + return false; + } + + switch (scheme) { + case "s3a": + // Only enable parallel read for S3, so far. + return true; + default: + return false; + } + } + + private void readChunkPages(Chunk chunk, BlockMetaData block) throws IOException { + if (fileDecryptor == null || fileDecryptor.plaintextFile()) { + currentRowGroup.addColumn(chunk.descriptor.col, chunk.readAllPages()); + return; + } + // Encrypted file + ColumnPath columnPath = ColumnPath.get(chunk.descriptor.col.getPath()); + InternalColumnDecryptionSetup columnDecryptionSetup = fileDecryptor.getColumnSetup(columnPath); + if (!columnDecryptionSetup.isEncrypted()) { // plaintext column + currentRowGroup.addColumn(chunk.descriptor.col, chunk.readAllPages()); + } else { // encrypted column + currentRowGroup.addColumn( + chunk.descriptor.col, + chunk.readAllPages( + columnDecryptionSetup.getMetaDataDecryptor(), + columnDecryptionSetup.getDataDecryptor(), + fileDecryptor.getFileAAD(), + block.getOrdinal(), + columnDecryptionSetup.getOrdinal())); + } + } + + private boolean advanceToNextBlock() { + if (currentBlock == blocks.size()) { + return false; + } + // update the current block and instantiate a dictionary reader for it + ++currentBlock; + return true; + } + + public long[] getRowIndices() { + long[] rowIndices = new long[blocks.size() * 2]; + for (int i = 0, n = blocks.size(); i < n; i++) { + BlockMetaData block = blocks.get(i); + rowIndices[i * 2] = getRowIndexOffset(block); + rowIndices[i * 2 + 1] = block.getRowCount(); + } + return rowIndices; + } + + // Uses reflection to get row index offset from a Parquet block metadata. + // + // The reason reflection is used here is that some Spark versions still depend on a + // Parquet version where the method `getRowIndexOffset` is not public. + private long getRowIndexOffset(BlockMetaData metaData) { + try { + Method method = BlockMetaData.class.getMethod("getRowIndexOffset"); + method.setAccessible(true); + return (long) method.invoke(metaData); + } catch (Exception e) { + throw new RuntimeException("Error when calling getRowIndexOffset", e); + } + } + + private RowRanges getRowRanges(int blockIndex) { + Preconditions.checkState( + FilterCompat.isFilteringRequired(options.getRecordFilter()), + "Should not be invoked if filter is null or NOOP"); + RowRanges rowRanges = blockRowRanges.get(blockIndex); + if (rowRanges == null) { + rowRanges = + ColumnIndexFilter.calculateRowRanges( + options.getRecordFilter(), + getColumnIndexReader(blockIndex), + paths.keySet(), + blocks.get(blockIndex).getRowCount()); + blockRowRanges.set(blockIndex, rowRanges); + } + return rowRanges; + } + + private static ParquetMetadata readFooter( + InputFile file, + ParquetReadOptions options, + SeekableInputStream f, + ParquetMetadataConverter converter) + throws IOException { + long fileLen = file.getLength(); + String filePath = file.toString(); + LOG.debug("File length {}", fileLen); + + int FOOTER_LENGTH_SIZE = 4; + + // MAGIC + data + footer + footerIndex + MAGIC + if (fileLen < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) { + throw new RuntimeException( + filePath + " is not a Parquet file (length is too low: " + fileLen + ")"); + } + + // Read footer length and magic string - with a single seek + byte[] magic = new byte[MAGIC.length]; + long fileMetadataLengthIndex = fileLen - magic.length - FOOTER_LENGTH_SIZE; + LOG.debug("reading footer index at {}", fileMetadataLengthIndex); + f.seek(fileMetadataLengthIndex); + int fileMetadataLength = BytesUtils.readIntLittleEndian(f); + f.readFully(magic); + + boolean encryptedFooterMode; + if (Arrays.equals(MAGIC, magic)) { + encryptedFooterMode = false; + } else if (Arrays.equals(EFMAGIC, magic)) { + encryptedFooterMode = true; + } else { + throw new RuntimeException( + filePath + + " is not a Parquet file. Expected magic number " + + "at tail, but found " + + Arrays.toString(magic)); + } + + long fileMetadataIndex = fileMetadataLengthIndex - fileMetadataLength; + LOG.debug("read footer length: {}, footer index: {}", fileMetadataLength, fileMetadataIndex); + if (fileMetadataIndex < magic.length || fileMetadataIndex >= fileMetadataLengthIndex) { + throw new RuntimeException( + "corrupted file: the footer index is not within the file: " + fileMetadataIndex); + } + f.seek(fileMetadataIndex); + + FileDecryptionProperties fileDecryptionProperties = options.getDecryptionProperties(); + InternalFileDecryptor fileDecryptor = null; + if (null != fileDecryptionProperties) { + fileDecryptor = new InternalFileDecryptor(fileDecryptionProperties); + } + + // Read all the footer bytes in one time to avoid multiple read operations, + // since it can be pretty time consuming for a single read operation in HDFS. + byte[] footerBytes = new byte[fileMetadataLength]; + f.readFully(footerBytes); + ByteBuffer footerBytesBuffer = ByteBuffer.wrap(footerBytes); + LOG.debug("Finished to read all footer bytes."); + InputStream footerBytesStream = ByteBufferInputStream.wrap(footerBytesBuffer); + + // Regular file, or encrypted file with plaintext footer + if (!encryptedFooterMode) { + return converter.readParquetMetadata( + footerBytesStream, options.getMetadataFilter(), fileDecryptor, false, fileMetadataLength); + } + + // Encrypted file with encrypted footer + if (fileDecryptor == null) { + throw new ParquetCryptoRuntimeException( + "Trying to read file with encrypted footer. " + "No keys available"); + } + FileCryptoMetaData fileCryptoMetaData = Util.readFileCryptoMetaData(footerBytesStream); + fileDecryptor.setFileCryptoMetaData( + fileCryptoMetaData.getEncryption_algorithm(), true, fileCryptoMetaData.getKey_metadata()); + // footer length is required only for signed plaintext footers + return converter.readParquetMetadata( + footerBytesStream, options.getMetadataFilter(), fileDecryptor, true, 0); + } + + private List filterRowGroups(List blocks) { + FilterCompat.Filter recordFilter = options.getRecordFilter(); + if (FilterCompat.isFilteringRequired(recordFilter)) { + // set up data filters based on configured levels + List levels = new ArrayList<>(); + + if (options.useStatsFilter()) { + levels.add(STATISTICS); + } + + if (options.useDictionaryFilter()) { + levels.add(DICTIONARY); + } + + if (options.useBloomFilter()) { + levels.add(BLOOMFILTER); + } + return RowGroupFilter.filterRowGroups(levels, recordFilter, blocks, this); + } + + return blocks; + } + + private static List listWithNulls(int size) { + return Stream.generate(() -> (T) null).limit(size).collect(Collectors.toList()); + } + + public void closeStream() throws IOException { + if (f != null) { + f.close(); + } + } + + @Override + public void close() throws IOException { + try { + if (f != null) { + f.close(); + } + } finally { + options.getCodecFactory().release(); + } + } + + /** + * Builder to concatenate the buffers of the discontinuous parts for the same column. These parts + * are generated as a result of the column-index based filtering when some pages might be skipped + * at reading. + */ + private class ChunkListBuilder { + private class ChunkData { + final List buffers = new ArrayList<>(); + OffsetIndex offsetIndex; + } + + private final Map map = new HashMap<>(); + + void add(ChunkDescriptor descriptor, List buffers) { + ChunkListBuilder.ChunkData data = map.get(descriptor); + if (data == null) { + data = new ChunkData(); + map.put(descriptor, data); + } + data.buffers.addAll(buffers); + } + + void setOffsetIndex(ChunkDescriptor descriptor, OffsetIndex offsetIndex) { + ChunkData data = map.get(descriptor); + if (data == null) { + data = new ChunkData(); + map.put(descriptor, data); + } + data.offsetIndex = offsetIndex; + } + + List build() { + List chunks = new ArrayList<>(); + for (Map.Entry entry : map.entrySet()) { + ChunkDescriptor descriptor = entry.getKey(); + ChunkData data = entry.getValue(); + chunks.add(new Chunk(descriptor, data.buffers, data.offsetIndex)); + } + return chunks; + } + } + + /** The data for a column chunk */ + private class Chunk { + private final ChunkDescriptor descriptor; + private final ByteBufferInputStream stream; + final OffsetIndex offsetIndex; + + /** + * @param descriptor descriptor for the chunk + * @param buffers ByteBuffers that contain the chunk + * @param offsetIndex the offset index for this column; might be null + */ + Chunk(ChunkDescriptor descriptor, List buffers, OffsetIndex offsetIndex) { + this.descriptor = descriptor; + this.stream = ByteBufferInputStream.wrap(buffers); + this.offsetIndex = offsetIndex; + } + + protected PageHeader readPageHeader(BlockCipher.Decryptor blockDecryptor, byte[] pageHeaderAAD) + throws IOException { + return Util.readPageHeader(stream, blockDecryptor, pageHeaderAAD); + } + + /** + * Calculate checksum of input bytes, throw decoding exception if it does not match the provided + * reference crc + */ + private void verifyCrc(int referenceCrc, byte[] bytes, String exceptionMsg) { + crc.reset(); + crc.update(bytes); + if (crc.getValue() != ((long) referenceCrc & 0xffffffffL)) { + throw new ParquetDecodingException(exceptionMsg); + } + } + + private ColumnPageReader readAllPages() throws IOException { + return readAllPages(null, null, null, -1, -1); + } + + private ColumnPageReader readAllPages( + BlockCipher.Decryptor headerBlockDecryptor, + BlockCipher.Decryptor pageBlockDecryptor, + byte[] aadPrefix, + int rowGroupOrdinal, + int columnOrdinal) + throws IOException { + List pagesInChunk = new ArrayList<>(); + DictionaryPage dictionaryPage = null; + PrimitiveType type = + fileMetaData.getSchema().getType(descriptor.col.getPath()).asPrimitiveType(); + + long valuesCountReadSoFar = 0; + int dataPageCountReadSoFar = 0; + byte[] dataPageHeaderAAD = null; + if (null != headerBlockDecryptor) { + dataPageHeaderAAD = + AesCipher.createModuleAAD( + aadPrefix, + ModuleCipherFactory.ModuleType.DataPageHeader, + rowGroupOrdinal, + columnOrdinal, + getPageOrdinal(dataPageCountReadSoFar)); + } + while (hasMorePages(valuesCountReadSoFar, dataPageCountReadSoFar)) { + byte[] pageHeaderAAD = dataPageHeaderAAD; + if (null != headerBlockDecryptor) { + // Important: this verifies file integrity (makes sure dictionary page had not been + // removed) + if (null == dictionaryPage && descriptor.metadata.hasDictionaryPage()) { + pageHeaderAAD = + AesCipher.createModuleAAD( + aadPrefix, + ModuleCipherFactory.ModuleType.DictionaryPageHeader, + rowGroupOrdinal, + columnOrdinal, + -1); + } else { + int pageOrdinal = getPageOrdinal(dataPageCountReadSoFar); + AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal); + } + } + + PageHeader pageHeader = readPageHeader(headerBlockDecryptor, pageHeaderAAD); + int uncompressedPageSize = pageHeader.getUncompressed_page_size(); + int compressedPageSize = pageHeader.getCompressed_page_size(); + final BytesInput pageBytes; + switch (pageHeader.type) { + case DICTIONARY_PAGE: + // there is only one dictionary page per column chunk + if (dictionaryPage != null) { + throw new ParquetDecodingException( + "more than one dictionary page in column " + descriptor.col); + } + pageBytes = this.readAsBytesInput(compressedPageSize); + if (options.usePageChecksumVerification() && pageHeader.isSetCrc()) { + verifyCrc( + pageHeader.getCrc(), + pageBytes.toByteArray(), + "could not verify dictionary page integrity, CRC checksum verification failed"); + } + DictionaryPageHeader dicHeader = pageHeader.getDictionary_page_header(); + dictionaryPage = + new DictionaryPage( + pageBytes, + uncompressedPageSize, + dicHeader.getNum_values(), + converter.getEncoding(dicHeader.getEncoding())); + // Copy crc to new page, used for testing + if (pageHeader.isSetCrc()) { + dictionaryPage.setCrc(pageHeader.getCrc()); + } + break; + + case DATA_PAGE: + DataPageHeader dataHeaderV1 = pageHeader.getData_page_header(); + pageBytes = this.readAsBytesInput(compressedPageSize); + if (options.usePageChecksumVerification() && pageHeader.isSetCrc()) { + verifyCrc( + pageHeader.getCrc(), + pageBytes.toByteArray(), + "could not verify page integrity, CRC checksum verification failed"); + } + DataPageV1 dataPageV1 = + new DataPageV1( + pageBytes, + dataHeaderV1.getNum_values(), + uncompressedPageSize, + converter.fromParquetStatistics( + getFileMetaData().getCreatedBy(), dataHeaderV1.getStatistics(), type), + converter.getEncoding(dataHeaderV1.getRepetition_level_encoding()), + converter.getEncoding(dataHeaderV1.getDefinition_level_encoding()), + converter.getEncoding(dataHeaderV1.getEncoding())); + // Copy crc to new page, used for testing + if (pageHeader.isSetCrc()) { + dataPageV1.setCrc(pageHeader.getCrc()); + } + pagesInChunk.add(dataPageV1); + valuesCountReadSoFar += dataHeaderV1.getNum_values(); + ++dataPageCountReadSoFar; + break; + + case DATA_PAGE_V2: + DataPageHeaderV2 dataHeaderV2 = pageHeader.getData_page_header_v2(); + int dataSize = + compressedPageSize + - dataHeaderV2.getRepetition_levels_byte_length() + - dataHeaderV2.getDefinition_levels_byte_length(); + pagesInChunk.add( + new DataPageV2( + dataHeaderV2.getNum_rows(), + dataHeaderV2.getNum_nulls(), + dataHeaderV2.getNum_values(), + this.readAsBytesInput(dataHeaderV2.getRepetition_levels_byte_length()), + this.readAsBytesInput(dataHeaderV2.getDefinition_levels_byte_length()), + converter.getEncoding(dataHeaderV2.getEncoding()), + this.readAsBytesInput(dataSize), + uncompressedPageSize, + converter.fromParquetStatistics( + getFileMetaData().getCreatedBy(), dataHeaderV2.getStatistics(), type), + dataHeaderV2.isIs_compressed())); + valuesCountReadSoFar += dataHeaderV2.getNum_values(); + ++dataPageCountReadSoFar; + break; + + default: + LOG.debug( + "skipping page of type {} of size {}", pageHeader.getType(), compressedPageSize); + stream.skipFully(compressedPageSize); + break; + } + } + if (offsetIndex == null && valuesCountReadSoFar != descriptor.metadata.getValueCount()) { + // Would be nice to have a CorruptParquetFileException or something as a subclass? + throw new IOException( + "Expected " + + descriptor.metadata.getValueCount() + + " values in column chunk at " + + file + + " offset " + + descriptor.metadata.getFirstDataPageOffset() + + " but got " + + valuesCountReadSoFar + + " values instead over " + + pagesInChunk.size() + + " pages ending at file offset " + + (descriptor.fileOffset + stream.position())); + } + CompressionCodecFactory.BytesInputDecompressor decompressor = + options.getCodecFactory().getDecompressor(descriptor.metadata.getCodec()); + return new ColumnPageReader( + decompressor, + pagesInChunk, + dictionaryPage, + offsetIndex, + blocks.get(currentBlock).getRowCount(), + pageBlockDecryptor, + aadPrefix, + rowGroupOrdinal, + columnOrdinal); + } + + private boolean hasMorePages(long valuesCountReadSoFar, int dataPageCountReadSoFar) { + return offsetIndex == null + ? valuesCountReadSoFar < descriptor.metadata.getValueCount() + : dataPageCountReadSoFar < offsetIndex.getPageCount(); + } + + private int getPageOrdinal(int dataPageCountReadSoFar) { + if (null == offsetIndex) { + return dataPageCountReadSoFar; + } + + return offsetIndex.getPageOrdinal(dataPageCountReadSoFar); + } + + /** + * @param size the size of the page + * @return the page + * @throws IOException if there is an error while reading from the file stream + */ + public BytesInput readAsBytesInput(int size) throws IOException { + return BytesInput.from(stream.sliceBuffers(size)); + } + } + + /** + * Describes a list of consecutive parts to be read at once. A consecutive part may contain whole + * column chunks or only parts of them (some pages). + */ + private class ConsecutivePartList { + private final long offset; + private final List chunks = new ArrayList<>(); + private long length; + private final SQLMetric fileReadTimeMetric; + private final SQLMetric fileReadSizeMetric; + private final SQLMetric readThroughput; + + /** + * @param offset where the first chunk starts + */ + ConsecutivePartList(long offset) { + if (metrics != null) { + this.fileReadTimeMetric = metrics.get("ParquetInputFileReadTime"); + this.fileReadSizeMetric = metrics.get("ParquetInputFileReadSize"); + this.readThroughput = metrics.get("ParquetInputFileReadThroughput"); + } else { + this.fileReadTimeMetric = null; + this.fileReadSizeMetric = null; + this.readThroughput = null; + } + this.offset = offset; + } + + /** + * Adds a chunk to the list. It must be consecutive to the previous chunk. + * + * @param descriptor a chunk descriptor + */ + public void addChunk(ChunkDescriptor descriptor) { + chunks.add(descriptor); + length += descriptor.size; + } + + private List allocateReadBuffers() { + int fullAllocations = Math.toIntExact(length / options.getMaxAllocationSize()); + int lastAllocationSize = Math.toIntExact(length % options.getMaxAllocationSize()); + + int numAllocations = fullAllocations + (lastAllocationSize > 0 ? 1 : 0); + List buffers = new ArrayList<>(numAllocations); + + for (int i = 0; i < fullAllocations; i += 1) { + buffers.add(options.getAllocator().allocate(options.getMaxAllocationSize())); + } + + if (lastAllocationSize > 0) { + buffers.add(options.getAllocator().allocate(lastAllocationSize)); + } + return buffers; + } + + /** + * @param f file to read the chunks from + * @param builder used to build chunk list to read the pages for the different columns + * @throws IOException if there is an error while reading from the stream + */ + public void readAll(SeekableInputStream f, ChunkListBuilder builder) throws IOException { + f.seek(offset); + + List buffers = allocateReadBuffers(); + long startNs = System.nanoTime(); + + for (ByteBuffer buffer : buffers) { + f.readFully(buffer); + buffer.flip(); + } + setReadMetrics(startNs); + + // report in a counter the data we just scanned + BenchmarkCounter.incrementBytesRead(length); + ByteBufferInputStream stream = ByteBufferInputStream.wrap(buffers); + for (int i = 0; i < chunks.size(); i++) { + ChunkDescriptor descriptor = chunks.get(i); + if (descriptor.col != null) { + builder.add(descriptor, stream.sliceBuffers(descriptor.size)); + } else { + stream.skipFully(descriptor.size); + } + } + } + + /** + * Api to read a consecutive range from the Parquet file in parallel. This is identical to + * {@link #readAll(SeekableInputStream, ChunkListBuilder) readAll}, except that the consecutive + * range is split into multiple smaller ranges and read in parallel. The parallelism can be set + * by specifying the threadpool size via {@link + * ReadOptions.Builder#withParallelIOThreadPoolSize(int)}. + * + * @param builder used to build chunk list to read the pages for the different columns + * @throws IOException if there is an error while reading from the stream + */ + public void readAllParallel(ChunkListBuilder builder) throws IOException { + + List buffers = allocateReadBuffers(); + long startNs = System.nanoTime(); + + int nThreads = cometOptions.parallelIOThreadPoolSize(); + ExecutorService threadPool = CometFileReaderThreadPool.getOrCreateThreadPool(nThreads); + List> futures = new ArrayList<>(); + + long currentOffset = this.offset; + int buffersPerThread = buffers.size() / nThreads; + int remaining = buffers.size() % nThreads; + // offset in input file each thread seeks to before beginning read + long[] offsets = new long[nThreads]; + // index of buffer where each thread will start writing data + int[] bufferIndexes = new int[nThreads]; + // number of buffers for each thread to fill + int[] numBuffers = new int[nThreads]; + + int bufferNum = 0; + for (int i = 0; i < nThreads; i++) { + int nBuffers = 0; + offsets[i] = currentOffset; + bufferIndexes[i] = bufferNum; + nBuffers = buffersPerThread; + for (int j = 0; j < buffersPerThread; j++) { + currentOffset += buffers.get(bufferNum).capacity(); + bufferNum++; + } + if (remaining > 0) { + remaining--; + currentOffset += buffers.get(bufferNum).capacity(); + bufferNum++; + nBuffers++; + } + numBuffers[i] = nBuffers; + } + for (int n = 0; n < nThreads; n++) { + int threadIndex = n; + long pos = offsets[threadIndex]; + int bufferIndex = bufferIndexes[threadIndex]; + int nBuffers = numBuffers[threadIndex]; + if (nBuffers == 0) { + continue; + } + + // Find the total number of bytes to read for the current thread + long tmp = 0; + for (int i = 0; i < nBuffers; i++) { + int bufNo = bufferIndex + i; + if (bufNo >= buffers.size()) break; + tmp += buffers.get(bufNo).capacity(); + } + final long length = tmp; + + futures.add( + threadPool.submit( + () -> { + SeekableInputStream inputStream = null; + try { + if (file instanceof CometInputFile) { + inputStream = (((CometInputFile) file).newStream(pos, length)); + } else { + inputStream = file.newStream(); + } + + inputStream.seek(pos); + long curPos = pos; + for (int i = 0; i < nBuffers; i++) { + int bufNo = bufferIndex + i; + if (bufNo >= buffers.size()) { + break; + } + ByteBuffer buffer = buffers.get(bufNo); + LOG.debug( + "Thread: {} Offset: {} Buffer: {} Size: {}", + threadIndex, + curPos, + bufNo, + buffer.capacity()); + curPos += buffer.capacity(); + inputStream.readFully(buffer); + buffer.flip(); + } // for + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + + return null; + })); + } + + for (Future future : futures) { + try { + future.get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } + + setReadMetrics(startNs); + + ByteBufferInputStream stream; + stream = ByteBufferInputStream.wrap(buffers); + // report in a counter the data we just scanned + BenchmarkCounter.incrementBytesRead(length); + for (int i = 0; i < chunks.size(); i++) { + ChunkDescriptor descriptor = chunks.get(i); + if (descriptor.col != null) { + builder.add(descriptor, stream.sliceBuffers(descriptor.size)); + } else { + stream.skipFully(descriptor.size); + } + } + } + + private void setReadMetrics(long startNs) { + long totalFileReadTimeNs = System.nanoTime() - startNs; + double sizeInMb = ((double) length) / (1024 * 1024); + double timeInSec = ((double) totalFileReadTimeNs) / 1000_0000_0000L; + double throughput = sizeInMb / timeInSec; + LOG.debug( + "Comet: File Read stats: Length: {} MB, Time: {} secs, throughput: {} MB/sec ", + sizeInMb, + timeInSec, + throughput); + if (fileReadTimeMetric != null) { + fileReadTimeMetric.add(totalFileReadTimeNs); + } + if (fileReadSizeMetric != null) { + fileReadSizeMetric.add(length); + } + if (readThroughput != null) { + readThroughput.set(throughput); + } + } + + /** + * @return the position following the last byte of these chunks + */ + public long endPos() { + return offset + length; + } + } + + /** Information needed to read a column chunk or a part of it. */ + private static class ChunkDescriptor { + private final ColumnDescriptor col; + private final ColumnChunkMetaData metadata; + private final long fileOffset; + private final long size; + + /** + * @param col column this chunk is part of + * @param metadata metadata for the column + * @param fileOffset offset in the file where this chunk starts + * @param size size of the chunk + */ + ChunkDescriptor( + ColumnDescriptor col, ColumnChunkMetaData metadata, long fileOffset, long size) { + this.col = col; + this.metadata = metadata; + this.fileOffset = fileOffset; + this.size = size; + } + + @Override + public int hashCode() { + return col.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } else if (obj instanceof ChunkDescriptor) { + return col.equals(((ChunkDescriptor) obj).col); + } else { + return false; + } + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/FooterReader.java b/common/src/main/java/org/apache/comet/parquet/FooterReader.java new file mode 100644 index 000000000..092eb442c --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/FooterReader.java @@ -0,0 +1,56 @@ +/* + * 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.comet.parquet; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.HadoopReadOptions; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.spark.sql.execution.datasources.PartitionedFile; + +/** + * Copied from Spark's `ParquetFooterReader` in order to avoid shading issue around Parquet. + * + *

`FooterReader` is a util class which encapsulates the helper methods of reading parquet file + * footer. + */ +public class FooterReader { + public static ParquetMetadata readFooter(Configuration configuration, PartitionedFile file) + throws IOException, URISyntaxException { + long start = file.start(); + long length = file.length(); + Path filePath = new Path(new URI(file.filePath().toString())); + CometInputFile inputFile = CometInputFile.fromPath(filePath, configuration); + ParquetReadOptions readOptions = + HadoopReadOptions.builder(inputFile.getConf(), inputFile.getPath()) + .withRange(start, start + length) + .build(); + ReadOptions cometReadOptions = ReadOptions.builder(configuration).build(); + // Use try-with-resources to ensure fd is closed. + try (FileReader fileReader = new FileReader(inputFile, readOptions, cometReadOptions)) { + return fileReader.getFooter(); + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/IndexFilter.java b/common/src/main/java/org/apache/comet/parquet/IndexFilter.java new file mode 100644 index 000000000..afa5687ca --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/IndexFilter.java @@ -0,0 +1,145 @@ +/* + * 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.comet.parquet; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.internal.filter2.columnindex.RowRanges; + +public class IndexFilter { + private final RowRanges rowRanges; + private final OffsetIndex offsetIndex; + private final long totalRowCount; + + public IndexFilter(RowRanges rowRanges, OffsetIndex offsetIndex, long totalRowCount) { + this.rowRanges = rowRanges; + this.offsetIndex = offsetIndex; + this.totalRowCount = totalRowCount; + } + + OffsetIndex filterOffsetIndex() { + List indexMap = new ArrayList<>(); + for (int i = 0, n = offsetIndex.getPageCount(); i < n; ++i) { + long from = offsetIndex.getFirstRowIndex(i); + if (rowRanges.isOverlapping(from, offsetIndex.getLastRowIndex(i, totalRowCount))) { + indexMap.add(i); + } + } + + int[] indexArray = new int[indexMap.size()]; + for (int i = 0; i < indexArray.length; i++) { + indexArray[i] = indexMap.get(i); + } + return new FilteredOffsetIndex(offsetIndex, indexArray); + } + + List calculateOffsetRanges(OffsetIndex filteredOffsetIndex, ColumnChunkMetaData cm) { + List ranges = new ArrayList<>(); + long firstPageOffset = offsetIndex.getOffset(0); + int n = filteredOffsetIndex.getPageCount(); + + if (n > 0) { + OffsetRange currentRange = null; + + // Add a range for the dictionary page if required + long rowGroupOffset = cm.getStartingPos(); + if (rowGroupOffset < firstPageOffset) { + currentRange = new OffsetRange(rowGroupOffset, (int) (firstPageOffset - rowGroupOffset)); + ranges.add(currentRange); + } + + for (int i = 0; i < n; ++i) { + long offset = filteredOffsetIndex.getOffset(i); + int length = filteredOffsetIndex.getCompressedPageSize(i); + if (currentRange == null || !currentRange.extend(offset, length)) { + currentRange = new OffsetRange(offset, length); + ranges.add(currentRange); + } + } + } + return ranges; + } + + private static class FilteredOffsetIndex implements OffsetIndex { + private final OffsetIndex offsetIndex; + private final int[] indexMap; + + private FilteredOffsetIndex(OffsetIndex offsetIndex, int[] indexMap) { + this.offsetIndex = offsetIndex; + this.indexMap = indexMap; + } + + @Override + public int getPageOrdinal(int pageIndex) { + return indexMap[pageIndex]; + } + + @Override + public int getPageCount() { + return indexMap.length; + } + + @Override + public long getOffset(int pageIndex) { + return offsetIndex.getOffset(indexMap[pageIndex]); + } + + @Override + public int getCompressedPageSize(int pageIndex) { + return offsetIndex.getCompressedPageSize(indexMap[pageIndex]); + } + + @Override + public long getFirstRowIndex(int pageIndex) { + return offsetIndex.getFirstRowIndex(indexMap[pageIndex]); + } + + @Override + public long getLastRowIndex(int pageIndex, long totalRowCount) { + int nextIndex = indexMap[pageIndex] + 1; + return (nextIndex >= offsetIndex.getPageCount() + ? totalRowCount + : offsetIndex.getFirstRowIndex(nextIndex)) + - 1; + } + } + + static class OffsetRange { + final long offset; + long length; + + private OffsetRange(long offset, int length) { + this.offset = offset; + this.length = length; + } + + private boolean extend(long offset, int length) { + if (this.offset + this.length == offset) { + this.length += length; + return true; + } else { + return false; + } + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java b/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java new file mode 100644 index 000000000..a15d84192 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java @@ -0,0 +1,186 @@ +/* + * 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.comet.parquet; + +import java.io.IOException; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.spark.sql.types.DataType; + +import org.apache.comet.vector.CometLazyVector; +import org.apache.comet.vector.CometVector; + +public class LazyColumnReader extends ColumnReader { + + // Remember the largest skipped index for sanity checking. + private int lastSkippedRowId = Integer.MAX_VALUE; + + // Track whether the underlying page is drained. + private boolean isPageDrained = true; + + // Leftover number of rows that did not skip in the previous batch. + private int numRowsToSkipFromPrevBatch; + + // The lazy vector being updated. + private final CometLazyVector vector; + + public LazyColumnReader( + DataType sparkReadType, + ColumnDescriptor descriptor, + int batchSize, + boolean useDecimal128, + boolean useLegacyDateTimestamp) { + super(sparkReadType, descriptor, batchSize, useDecimal128, useLegacyDateTimestamp); + this.batchSize = 0; // the batch size is set later in `readBatch` + this.vector = new CometLazyVector(sparkReadType, this, useDecimal128); + } + + @Override + public void setPageReader(PageReader pageReader) throws IOException { + super.setPageReader(pageReader); + lastSkippedRowId = Integer.MAX_VALUE; + isPageDrained = true; + numRowsToSkipFromPrevBatch = 0; + currentNumValues = batchSize; + } + + /** + * Lazily read a batch of 'total' rows for this column. The includes: 1) Skip any unused rows from + * the previous batch 2) Reset the native columnar batch 3) Reset tracking variables + * + * @param total the number of rows in the batch. MUST be <= the number of rows available in this + * column chunk. + */ + @Override + public void readBatch(int total) { + // Before starting a new batch, take care of the remaining rows to skip from the previous batch. + tryPageSkip(batchSize); + numRowsToSkipFromPrevBatch += batchSize - currentNumValues; + + // Now first reset the current columnar batch so that it can be used to fill in a new batch + // of values. Then, keep reading more data pages (via 'readBatch') until the current batch is + // full, or we have read 'total' number of values. + Native.resetBatch(nativeHandle); + + batchSize = total; + currentNumValues = 0; + lastSkippedRowId = -1; + } + + @Override + public CometVector currentBatch() { + return vector; + } + + /** Read all rows up to the `batchSize`. Expects no rows are skipped so far. */ + public void readAllBatch() { + // All rows should be read without any skips so far + assert (lastSkippedRowId == -1); + + readBatch(batchSize - 1, 0); + } + + /** + * Read at least up to `rowId`. It may read beyond `rowId` if enough rows available in the page. + * It may skip reading rows before `rowId`. In case `rowId` is already read, return immediately. + * + * @param rowId the row index in the batch to read. + * @return true if `rowId` is newly materialized, or false if `rowId` is already materialized. + */ + public boolean materializeUpToIfNecessary(int rowId) { + // Not allowed reading rowId if it may have skipped previously. + assert (rowId > lastSkippedRowId); + + // If `rowId` is already materialized, return immediately. + if (rowId < currentNumValues) return false; + + int numRowsWholePageSkipped = tryPageSkip(rowId); + readBatch(rowId, numRowsWholePageSkipped); + return true; + } + + /** + * Read up to `rowId` (inclusive). If the whole pages are skipped previously in `tryPageSkip()`, + * pad the number of whole page skipped rows with nulls to the underlying vector before reading. + * + * @param rowId the row index in the batch to read. + * @param numNullRowsToPad the number of nulls to pad before reading. + */ + private void readBatch(int rowId, int numNullRowsToPad) { + if (numRowsToSkipFromPrevBatch > 0) { + // Reaches here only when starting a new batch and the page is previously drained + readPage(); + isPageDrained = false; + Native.skipBatch(nativeHandle, numRowsToSkipFromPrevBatch, true); + numRowsToSkipFromPrevBatch = 0; + } + while (rowId >= currentNumValues) { + int numRowsToRead = batchSize - currentNumValues; + if (isPageDrained) { + readPage(); + } + int[] array = Native.readBatch(nativeHandle, numRowsToRead, numNullRowsToPad); + int read = array[0]; + isPageDrained = read < numRowsToRead; + currentNumValues += read; + currentNumNulls += array[1]; + // No need to update numNullRowsToPad. numNullRowsToPad > 0 means there were whole page skips. + // That guarantees that the Native.readBatch can read up to rowId in the current page. + } + } + + /** + * Try to skip until `rowId` (exclusive). If possible, it skips whole underlying pages without + * decompressing. In that case, it returns early at the page end, so that the next iteration can + * lazily decide to `readPage()` or `tryPageSkip()` again. + * + * @param rowId the row index in the batch that it tries to skip up until (exclusive). + * @return the number of rows that the whole page skips were applied. + */ + private int tryPageSkip(int rowId) { + int total = rowId - currentNumValues; + int wholePageSkipped = 0; + if (total > 0) { + // First try to skip from the non-drained underlying page. + int skipped = isPageDrained ? 0 : Native.skipBatch(nativeHandle, total); + total -= skipped; + isPageDrained = total > 0; + if (isPageDrained) { + ColumnPageReader columnPageReader = (ColumnPageReader) pageReader; + // It is always `columnPageReader.getPageValueCount() > numRowsToSkipFromPriorBatch` + int pageValueCount = columnPageReader.getPageValueCount() - numRowsToSkipFromPrevBatch; + while (pageValueCount <= total) { + // skip the entire page if the next page is small enough + columnPageReader.skipPage(); + numRowsToSkipFromPrevBatch = 0; + total -= pageValueCount; + wholePageSkipped += pageValueCount; + pageValueCount = columnPageReader.getPageValueCount(); + } + } + + currentNumValues += skipped + wholePageSkipped; + currentNumNulls += skipped; + lastSkippedRowId = currentNumValues - 1; + } + return wholePageSkipped; + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java b/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java new file mode 100644 index 000000000..b8722ca78 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java @@ -0,0 +1,80 @@ +/* + * 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.comet.parquet; + +import org.apache.arrow.c.ArrowArray; +import org.apache.arrow.c.ArrowSchema; +import org.apache.arrow.c.Data; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.FieldVector; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.spark.sql.types.DataType; + +import org.apache.comet.vector.CometPlainVector; +import org.apache.comet.vector.CometVector; + +/** A metadata column reader that can be extended by {@link RowIndexColumnReader} etc. */ +public class MetadataColumnReader extends AbstractColumnReader { + private final BufferAllocator allocator = new RootAllocator(); + private CometVector vector; + + public MetadataColumnReader(DataType type, ColumnDescriptor descriptor, boolean useDecimal128) { + // TODO: should we handle legacy dates & timestamps for metadata columns? + super(type, descriptor, useDecimal128, false); + } + + @Override + public void setBatchSize(int batchSize) { + close(); + super.setBatchSize(batchSize); + } + + @Override + public void readBatch(int total) { + if (vector == null) { + long[] addresses = Native.currentBatch(nativeHandle); + try (ArrowArray array = ArrowArray.wrap(addresses[0]); + ArrowSchema schema = ArrowSchema.wrap(addresses[1])) { + FieldVector fieldVector = Data.importVector(allocator, array, schema, null); + vector = new CometPlainVector(fieldVector, useDecimal128); + } + } + vector.setNumValues(total); + } + + void setNumNulls(int total) { + vector.setNumNulls(total); + } + + @Override + public CometVector currentBatch() { + return vector; + } + + @Override + public void close() { + if (vector != null) { + vector.close(); + vector = null; + } + super.close(); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/Native.java b/common/src/main/java/org/apache/comet/parquet/Native.java new file mode 100644 index 000000000..0887ae12f --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/Native.java @@ -0,0 +1,235 @@ +/* + * 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.comet.parquet; + +import java.nio.ByteBuffer; + +import org.apache.comet.NativeBase; + +public final class Native extends NativeBase { + public static int[] readBatch(long handle, int batchSize) { + return readBatch(handle, batchSize, 0); + } + + public static int skipBatch(long handle, int batchSize) { + return skipBatch(handle, batchSize, false); + } + + /** Native APIs * */ + + /** + * Creates a reader for a primitive Parquet column. + * + * @param physicalTypeId id for Parquet physical type + * @param logicalTypeId id for Parquet logical type + * @param expectedPhysicalTypeId id for Parquet physical type, converted from Spark read type. + * This is used for type promotion. + * @param path the path from the root schema to the column, derived from the method + * 'ColumnDescriptor#getPath()'. + * @param maxDl the maximum definition level of the primitive column + * @param maxRl the maximum repetition level of the primitive column + * @param bitWidth (only set when logical type is INT) the bit width for the integer type (INT8, + * INT16, INT32, etc) + * @param isSigned (only set when logical type is INT) whether it is signed or unsigned int. + * @param typeLength number of bytes required to store a value of the type, only set when the + * physical type is FIXED_LEN_BYTE_ARRAY, otherwise it's 0. + * @param precision (only set when logical type is DECIMAL) precision of the decimal type + * @param expectedPrecision (only set when logical type is DECIMAL) precision of the decimal type + * from Spark read schema. This is used for type promotion. + * @param scale (only set when logical type is DECIMAL) scale of the decimal type + * @param tu (only set when logical type is TIMESTAMP) unit for the timestamp + * @param isAdjustedUtc (only set when logical type is TIMESTAMP) whether the timestamp is + * adjusted to UTC or not + * @param batchSize the batch size for the columnar read + * @param useDecimal128 whether to always return 128 bit decimal regardless of precision + * @param useLegacyDateTimestampOrNTZ whether to read legacy dates/timestamps as it is + * @return a pointer to a native Parquet column reader created + */ + public static native long initColumnReader( + int physicalTypeId, + int logicalTypeId, + int expectedPhysicalTypeId, + String[] path, + int maxDl, + int maxRl, + int bitWidth, + boolean isSigned, + int typeLength, + int precision, + int expectedPrecision, + int scale, + int tu, + boolean isAdjustedUtc, + int batchSize, + boolean useDecimal128, + boolean useLegacyDateTimestampOrNTZ); + + /** + * Pass a Parquet dictionary page to the native column reader. Note this should only be called + * once per Parquet column chunk. Otherwise it'll panic. + * + * @param handle the handle to the native Parquet column reader + * @param dictionaryValueCount the number of values in this dictionary + * @param dictionaryData the actual dictionary page data, including repetition/definition levels + * as well as values + * @param encoding the encoding used by the dictionary + */ + public static native void setDictionaryPage( + long handle, int dictionaryValueCount, byte[] dictionaryData, int encoding); + + /** + * Passes a Parquet data page V1 to the native column reader. + * + * @param handle the handle to the native Parquet column reader + * @param pageValueCount the number of values in this data page + * @param pageData the actual page data, which should only contain PLAIN-encoded values. + * @param valueEncoding the encoding used by the values + */ + public static native void setPageV1( + long handle, int pageValueCount, byte[] pageData, int valueEncoding); + + /** + * Passes a Parquet data page V1 to the native column reader. + * + * @param handle the handle to the native Parquet column reader + * @param pageValueCount the number of values in this data page + * @param buffer the actual page data, represented by a DirectByteBuffer. + * @param valueEncoding the encoding used by the values + */ + public static native void setPageBufferV1( + long handle, int pageValueCount, ByteBuffer buffer, int valueEncoding); + + /** + * Passes a Parquet data page V2 to the native column reader. + * + * @param handle the handle to the native Parquet column reader + * @param pageValueCount the number of values in this data page + * @param defLevelData the data for definition levels + * @param repLevelData the data for repetition levels + * @param valueData the data for values + * @param valueEncoding the encoding used by the values + */ + public static native void setPageV2( + long handle, + int pageValueCount, + byte[] defLevelData, + byte[] repLevelData, + byte[] valueData, + int valueEncoding); + + /** + * Reset the current columnar batch. This will clear all the content of the batch as well as any + * internal state such as the current offset. + * + * @param handle the handle to the native Parquet column reader + */ + public static native void resetBatch(long handle); + + /** + * Reads at most 'batchSize' number of rows from the native Parquet column reader. Returns a tuple + * where the first element is the actual number of rows read (including both nulls and non-nulls), + * and the second element is the number of nulls read. + * + *

If the returned value is < 'batchSize' then it means the current page has been completely + * drained. In this case, the caller should call {@link Native#setPageV1} or {@link + * Native#setPageV2} before the next 'readBatch' call. + * + *

Note that the current page could also be drained if the returned value = 'batchSize', i.e., + * the remaining number of rows in the page is exactly equal to 'batchSize'. In this case, the + * next 'readBatch' call will return 0 and the caller should call {@link Native#setPageV1} or + * {@link Native#setPageV2} next. + * + *

If `nullPadSize` > 0, it pads nulls into the underlying vector before the values will be + * read into. + * + * @param handle the handle to the native Parquet column reader + * @param batchSize the number of rows to be read + * @param nullPadSize the number of nulls to pad before reading. + * @return a tuple: (the actual number of rows read, the number of nulls read) + */ + public static native int[] readBatch(long handle, int batchSize, int nullPadSize); + + /** + * Skips at most 'batchSize' number of rows from the native Parquet column reader, and returns the + * actual number of rows skipped. + * + *

If the returned value is < 'batchSize' then it means the current page has been completely + * drained. In this case, the caller should call {@link Native#setPageV1} or {@link + * Native#setPageV2} before the next 'skipBatch' call. + * + *

Note that the current page could also be drained if the returned value = 'batchSize', i.e., + * the remaining number of rows in the page is exactly equal to 'batchSize'. In this case, the + * next 'skipBatch' call will return 0 and the caller should call {@link Native#setPageV1} or + * {@link Native#setPageV2} next. + * + * @param handle the handle to the native Parquet column reader + * @param batchSize the number of rows to skip in the current page + * @param discard if true, discard read rows without padding nulls into the underlying vector + * @return the actual number of rows skipped + */ + public static native int skipBatch(long handle, int batchSize, boolean discard); + + /** + * Returns the current batch constructed via 'readBatch' + * + * @param handle the handle to the native Parquet column reader + * @return a long array with 2 elements, the first is the address to native Arrow array, and the + * second is the address to the Arrow schema. + */ + public static native long[] currentBatch(long handle); + + /** Set methods to set a constant value for the reader, so it'll return constant vectors */ + public static native void setNull(long handle); + + public static native void setBoolean(long handle, boolean value); + + public static native void setByte(long handle, byte value); + + public static native void setShort(long handle, short value); + + public static native void setInt(long handle, int value); + + public static native void setLong(long handle, long value); + + public static native void setFloat(long handle, float value); + + public static native void setDouble(long handle, double value); + + public static native void setBinary(long handle, byte[] value); + + /** Set decimal backed by FixedLengthByteArray */ + public static native void setDecimal(long handle, byte[] value); + + /** Set position of row index vector for Iceberg Metadata Column */ + public static native void setPosition(long handle, long value, int size); + + /** Set row index vector for Spark row index metadata column and return vector size */ + public static native int setIndices(long handle, long offset, int size, long[] indices); + + /** Set deleted info for Iceberg Metadata Column */ + public static native void setIsDeleted(long handle, boolean[] isDeleted); + + /** + * Closes the native Parquet column reader and releases all resources associated with it. + * + * @param handle the handle to the native Parquet column reader + */ + public static native void closeColumnReader(long handle); +} diff --git a/common/src/main/java/org/apache/comet/parquet/ReadOptions.java b/common/src/main/java/org/apache/comet/parquet/ReadOptions.java new file mode 100644 index 000000000..6754443e6 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/ReadOptions.java @@ -0,0 +1,197 @@ +/* + * 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.comet.parquet; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.spark.SparkEnv; +import org.apache.spark.launcher.SparkLauncher; + +/** + * Comet specific Parquet related read options. + * + *

TODO: merge this with {@link org.apache.parquet.HadoopReadOptions} once PARQUET-2203 is done. + */ +public class ReadOptions { + private static final Logger LOG = LoggerFactory.getLogger(ReadOptions.class); + public static final String COMET_PARQUET_PARALLEL_IO_ENABLED = + "comet.parquet.read.parallel.io.enabled"; + public static final boolean COMET_PARQUET_PARALLEL_IO_ENABLED_DEFAULT = true; + + public static final String COMET_PARQUET_PARALLEL_IO_THREADS = + "comet.parquet.read.parallel.io.thread-pool.size"; + public static final int COMET_PARQUET_PARALLEL_IO_THREADS_DEFAULT = 32; + + public static final String COMET_IO_MERGE_RANGES = "comet.parquet.read.io.mergeRanges"; + private static final boolean COMET_IO_MERGE_RANGES_DEFAULT = true; + + public static final String COMET_IO_MERGE_RANGES_DELTA = + "comet.parquet.read.io.mergeRanges.delta"; + private static final int COMET_IO_MERGE_RANGES_DELTA_DEFAULT = 1 << 23; // 8 MB + + // Max number of concurrent tasks we expect. Used to autoconfigure S3 client connections + public static final int S3A_MAX_EXPECTED_PARALLELISM = 32; + // defined in hadoop-aws - org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS + public static final String S3A_MAXIMUM_CONNECTIONS = "fs.s3a.connection.maximum"; + // default max connections in S3A - org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAXIMUM_CONNECTIONS + public static final int S3A_DEFAULT_MAX_HTTP_CONNECTIONS = 96; + + public static final String S3A_READAHEAD_RANGE = "fs.s3a.readahead.range"; + // Default read ahead range in Hadoop is 64K; we increase it to 1 MB + public static final long COMET_DEFAULT_READAHEAD_RANGE = 1 * 1024 * 1024; // 1 MB + + private final boolean parallelIOEnabled; + private final int parallelIOThreadPoolSize; + private final boolean ioMergeRanges; + private final int ioMergeRangesDelta; + + ReadOptions( + boolean parallelIOEnabled, + int parallelIOThreadPoolSize, + boolean ioMergeRanges, + int ioMergeRangesDelta) { + this.parallelIOEnabled = parallelIOEnabled; + this.parallelIOThreadPoolSize = parallelIOThreadPoolSize; + this.ioMergeRanges = ioMergeRanges; + this.ioMergeRangesDelta = ioMergeRangesDelta; + } + + public boolean isParallelIOEnabled() { + return this.parallelIOEnabled; + } + + public int parallelIOThreadPoolSize() { + return this.parallelIOThreadPoolSize; + } + + public boolean isIOMergeRangesEnabled() { + return ioMergeRanges; + } + + public int getIOMergeRangesDelta() { + return ioMergeRangesDelta; + } + + public static Builder builder(Configuration conf) { + return new Builder(conf); + } + + public static Builder builder() { + return builder(new Configuration()); + } + + public static class Builder { + private final Configuration conf; + + private boolean parallelIOEnabled; + private int parallelIOThreadPoolSize; + private boolean ioMergeRanges; + private int ioMergeRangesDelta; + + /** + * Whether to enable Parquet parallel IO when reading row groups. If true, Parquet reader will + * use multiple threads to read multiple chunks of data from the current row group in parallel. + */ + public Builder enableParallelIO(boolean b) { + this.parallelIOEnabled = b; + return this; + } + + /** + * Specify the number of threads to be used in parallel IO. + * + *

Note: this will only be effective if parallel IO is enabled (e.g., via {@link + * #enableParallelIO(boolean)}). + */ + public Builder withParallelIOThreadPoolSize(int numThreads) { + this.parallelIOThreadPoolSize = numThreads; + return this; + } + + public Builder enableIOMergeRanges(boolean enableIOMergeRanges) { + this.ioMergeRanges = enableIOMergeRanges; + return this; + } + + public Builder withIOMergeRangesDelta(int ioMergeRangesDelta) { + this.ioMergeRangesDelta = ioMergeRangesDelta; + return this; + } + + public ReadOptions build() { + return new ReadOptions( + parallelIOEnabled, parallelIOThreadPoolSize, ioMergeRanges, ioMergeRangesDelta); + } + + public Builder(Configuration conf) { + this.conf = conf; + this.parallelIOEnabled = + conf.getBoolean( + COMET_PARQUET_PARALLEL_IO_ENABLED, COMET_PARQUET_PARALLEL_IO_ENABLED_DEFAULT); + this.parallelIOThreadPoolSize = + conf.getInt(COMET_PARQUET_PARALLEL_IO_THREADS, COMET_PARQUET_PARALLEL_IO_THREADS_DEFAULT); + this.ioMergeRanges = conf.getBoolean(COMET_IO_MERGE_RANGES, COMET_IO_MERGE_RANGES_DEFAULT); + this.ioMergeRangesDelta = + conf.getInt(COMET_IO_MERGE_RANGES_DELTA, COMET_IO_MERGE_RANGES_DELTA_DEFAULT); + // override some S3 defaults + setS3Config(); + } + + // For paths to S3, if the s3 connection pool max is less than twice the product of + // parallel reader threads * number of cores, then increase the connection pool max + private void setS3Config() { + int s3ConnectionsMax = S3A_DEFAULT_MAX_HTTP_CONNECTIONS; + SparkEnv env = SparkEnv.get(); + // Use a default number of cores in case we are using the FileReader outside the context + // of Spark. + int numExecutorCores = S3A_MAX_EXPECTED_PARALLELISM; + if (env != null) { + numExecutorCores = env.conf().getInt(SparkLauncher.EXECUTOR_CORES, numExecutorCores); + } + int parallelReaderThreads = this.parallelIOEnabled ? this.parallelIOThreadPoolSize : 1; + s3ConnectionsMax = Math.max(numExecutorCores * parallelReaderThreads * 2, s3ConnectionsMax); + + setS3ConfIfGreater(conf, S3A_MAXIMUM_CONNECTIONS, s3ConnectionsMax); + setS3ConfIfGreater(conf, S3A_READAHEAD_RANGE, COMET_DEFAULT_READAHEAD_RANGE); + } + + // Update the conf iff the new value is greater than the existing val + private void setS3ConfIfGreater(Configuration conf, String key, int newVal) { + int maxVal = newVal; + String curr = conf.get(key); + if (curr != null && !curr.isEmpty()) { + maxVal = Math.max(Integer.parseInt(curr), newVal); + } + LOG.info("File reader auto configured '{}={}'", key, maxVal); + conf.set(key, Integer.toString(maxVal)); + } + + // Update the conf iff the new value is greater than the existing val. This handles values that + // may have suffixes (K, M, G, T, P, E) indicating well known bytes size suffixes + private void setS3ConfIfGreater(Configuration conf, String key, long newVal) { + long maxVal = conf.getLongBytes(key, newVal); + maxVal = Math.max(maxVal, newVal); + LOG.info("File reader auto configured '{}={}'", key, maxVal); + conf.set(key, Long.toString(maxVal)); + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/RowGroupFilter.java b/common/src/main/java/org/apache/comet/parquet/RowGroupFilter.java new file mode 100644 index 000000000..40a61c05e --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/RowGroupFilter.java @@ -0,0 +1,113 @@ +/* + * 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.comet.parquet; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.parquet.filter2.compat.FilterCompat; +import org.apache.parquet.filter2.compat.FilterCompat.Filter; +import org.apache.parquet.filter2.compat.FilterCompat.NoOpFilter; +import org.apache.parquet.filter2.compat.FilterCompat.Visitor; +import org.apache.parquet.filter2.dictionarylevel.DictionaryFilter; +import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.filter2.predicate.SchemaCompatibilityValidator; +import org.apache.parquet.filter2.statisticslevel.StatisticsFilter; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.schema.MessageType; + +public class RowGroupFilter implements Visitor> { + private final List blocks; + private final MessageType schema; + private final List levels; + private final FileReader reader; + + public enum FilterLevel { + STATISTICS, + DICTIONARY, + BLOOMFILTER + } + + public static List filterRowGroups( + List levels, Filter filter, List blocks, FileReader reader) { + return filter.accept(new RowGroupFilter(levels, blocks, reader)); + } + + private RowGroupFilter(List levels, List blocks, FileReader reader) { + this.levels = levels; + this.blocks = blocks; + this.reader = reader; + this.schema = reader.getFileMetaData().getSchema(); + } + + @Override + public List visit(FilterCompat.FilterPredicateCompat filterPredicateCompat) { + FilterPredicate filterPredicate = filterPredicateCompat.getFilterPredicate(); + + // check that the schema of the filter matches the schema of the file + SchemaCompatibilityValidator.validate(filterPredicate, schema); + + List filteredBlocks = new ArrayList<>(); + + for (BlockMetaData block : blocks) { + boolean drop = false; + + if (levels.contains(FilterLevel.STATISTICS)) { + drop = StatisticsFilter.canDrop(filterPredicate, block.getColumns()); + } + + if (!drop && levels.contains(FilterLevel.DICTIONARY)) { + drop = + DictionaryFilter.canDrop( + filterPredicate, + block.getColumns(), + new DictionaryPageReader( + block, + reader.getFileMetaData().getFileDecryptor(), + reader.getInputStream(), + reader.getOptions())); + } + + if (!drop && levels.contains(FilterLevel.BLOOMFILTER)) { + drop = + filterPredicate.accept( + new BloomFilterReader( + block, reader.getFileMetaData().getFileDecryptor(), reader.getInputStream())); + } + + if (!drop) { + filteredBlocks.add(block); + } + } + + return filteredBlocks; + } + + @Override + public List visit( + FilterCompat.UnboundRecordFilterCompat unboundRecordFilterCompat) { + return blocks; + } + + @Override + public List visit(NoOpFilter noOpFilter) { + return blocks; + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java b/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java new file mode 100644 index 000000000..d5d73b078 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java @@ -0,0 +1,72 @@ +/* + * 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.comet.parquet; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.PrimitiveIterator; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.internal.filter2.columnindex.RowRanges; + +public class RowGroupReader implements PageReadStore { + private final Map readers = new HashMap<>(); + private final long rowCount; + private final RowRanges rowRanges; + + public RowGroupReader(long rowCount) { + this.rowCount = rowCount; + this.rowRanges = null; + } + + RowGroupReader(RowRanges rowRanges) { + this.rowRanges = rowRanges; + this.rowCount = rowRanges.rowCount(); + } + + @Override + public long getRowCount() { + return rowCount; + } + + @Override + public PageReader getPageReader(ColumnDescriptor path) { + final PageReader pageReader = readers.get(path); + if (pageReader == null) { + throw new IllegalArgumentException( + path + " is not found: " + readers.keySet() + " " + rowCount); + } + return pageReader; + } + + @Override + public Optional getRowIndexes() { + return rowRanges == null ? Optional.empty() : Optional.of(rowRanges.iterator()); + } + + void addColumn(ColumnDescriptor path, ColumnPageReader reader) { + if (readers.put(path, reader) != null) { + throw new IllegalStateException(path + " was already added"); + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java b/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java new file mode 100644 index 000000000..8448318db --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java @@ -0,0 +1,49 @@ +/* + * 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.comet.parquet; + +import org.apache.spark.sql.types.*; + +/** + * A column reader that returns the row index vector. Used for reading row index metadata column for + * Spark 3.4+. The row index can be accessed by {@code _tmp_metadata_row_index} column. + */ +public class RowIndexColumnReader extends MetadataColumnReader { + /** The row indices that are used to initialize this column reader. */ + private final long[] indices; + + /** The current number of indices to skip reading from {@code indices}. */ + private long offset; + + public RowIndexColumnReader(StructField field, int batchSize, long[] indices) { + super(field.dataType(), TypeUtil.convertToParquet(field), false); + this.indices = indices; + setBatchSize(batchSize); + } + + @Override + public void readBatch(int total) { + Native.resetBatch(nativeHandle); + int count = Native.setIndices(nativeHandle, offset, total, indices); + offset += count; + + super.readBatch(count); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/SupportsComet.java b/common/src/main/java/org/apache/comet/parquet/SupportsComet.java new file mode 100644 index 000000000..f330a7681 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/SupportsComet.java @@ -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.comet.parquet; + +public interface SupportsComet { + boolean isCometEnabled(); +} diff --git a/common/src/main/java/org/apache/comet/parquet/TypeUtil.java b/common/src/main/java/org/apache/comet/parquet/TypeUtil.java new file mode 100644 index 000000000..340bf9823 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/TypeUtil.java @@ -0,0 +1,279 @@ +/* + * 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.comet.parquet; + +import java.util.Arrays; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.LogicalTypeAnnotation.*; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; +import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException; +import org.apache.spark.sql.types.*; + +import org.apache.comet.CometConf; + +public class TypeUtil { + + /** Converts the input Spark 'field' into a Parquet column descriptor. */ + public static ColumnDescriptor convertToParquet(StructField field) { + Type.Repetition repetition; + int maxDefinitionLevel; + if (field.nullable()) { + repetition = Type.Repetition.OPTIONAL; + maxDefinitionLevel = 1; + } else { + repetition = Type.Repetition.REQUIRED; + maxDefinitionLevel = 0; + } + String[] path = new String[] {field.name()}; + + DataType type = field.dataType(); + + Types.PrimitiveBuilder builder = null; + // Only partition column can be `NullType`, which also uses `ConstantColumnReader`. Here we + // piggy-back onto Parquet boolean type for constant vector of null values, we don't really + // care what Parquet type it is. + if (type == DataTypes.BooleanType || type == DataTypes.NullType) { + builder = Types.primitive(PrimitiveType.PrimitiveTypeName.BOOLEAN, repetition); + } else if (type == DataTypes.IntegerType || type instanceof YearMonthIntervalType) { + builder = Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition); + } else if (type == DataTypes.DateType) { + builder = + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition) + .as(LogicalTypeAnnotation.dateType()); + } else if (type == DataTypes.ByteType) { + builder = + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition) + .as(LogicalTypeAnnotation.intType(8, true)); + } else if (type == DataTypes.ShortType) { + builder = + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition) + .as(LogicalTypeAnnotation.intType(16, true)); + } else if (type == DataTypes.LongType) { + builder = Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition); + } else if (type == DataTypes.BinaryType) { + builder = Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, repetition); + } else if (type == DataTypes.StringType) { + builder = + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, repetition) + .as(LogicalTypeAnnotation.stringType()); + } else if (type == DataTypes.FloatType) { + builder = Types.primitive(PrimitiveType.PrimitiveTypeName.FLOAT, repetition); + } else if (type == DataTypes.DoubleType) { + builder = Types.primitive(PrimitiveType.PrimitiveTypeName.DOUBLE, repetition); + } else if (type == DataTypes.TimestampType) { + builder = + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition) + .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MICROS)); + } else if (type == TimestampNTZType$.MODULE$) { + builder = + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition) + .as(LogicalTypeAnnotation.timestampType(false, TimeUnit.MICROS)); + } else if (type instanceof DecimalType) { + DecimalType decimalType = (DecimalType) type; + builder = + Types.primitive(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, repetition) + .length(16) // always store as Decimal128 + .as(LogicalTypeAnnotation.decimalType(decimalType.scale(), decimalType.precision())); + } + if (builder == null) { + throw new UnsupportedOperationException("Unsupported input Spark type: " + type); + } + + return new ColumnDescriptor(path, builder.named(field.name()), 0, maxDefinitionLevel); + } + + /** + * Check whether the Parquet 'descriptor' and Spark read type 'sparkType' are compatible. If not, + * throw exception. + * + *

This mostly follows the logic in Spark's + * ParquetVectorUpdaterFactory#getUpdater(ColumnDescriptor, DataType) + * + * @param descriptor descriptor for a Parquet primitive column + * @param sparkType Spark read type + */ + public static void checkParquetType(ColumnDescriptor descriptor, DataType sparkType) { + PrimitiveType.PrimitiveTypeName typeName = descriptor.getPrimitiveType().getPrimitiveTypeName(); + LogicalTypeAnnotation logicalTypeAnnotation = + descriptor.getPrimitiveType().getLogicalTypeAnnotation(); + boolean allowTypePromotion = (boolean) CometConf.COMET_SCHEMA_EVOLUTION_ENABLED().get(); + + if (sparkType instanceof NullType) { + return; + } + + switch (typeName) { + case BOOLEAN: + if (sparkType == DataTypes.BooleanType) return; + break; + case INT32: + if (sparkType == DataTypes.IntegerType || canReadAsIntDecimal(descriptor, sparkType)) { + return; + } else if (sparkType == DataTypes.LongType + && isUnsignedIntTypeMatched(logicalTypeAnnotation, 32)) { + // In `ParquetToSparkSchemaConverter`, we map parquet UINT32 to our LongType. + // For unsigned int32, it stores as plain signed int32 in Parquet when dictionary + // fallbacks. We read them as long values. + return; + } else if (sparkType == DataTypes.LongType && allowTypePromotion) { + // In Comet we allow schema evolution from int to long, if + // `spark.comet.schemaEvolution.enabled` is enabled. + return; + } else if (sparkType == DataTypes.ByteType || sparkType == DataTypes.ShortType) { + return; + } else if (sparkType == DataTypes.DateType) { + // TODO: use dateTimeRebaseMode from Spark side + return; + } else if (sparkType instanceof YearMonthIntervalType) { + return; + } + break; + case INT64: + if (sparkType == DataTypes.LongType || canReadAsLongDecimal(descriptor, sparkType)) { + return; + } else if (isLongDecimal(sparkType) + && isUnsignedIntTypeMatched(logicalTypeAnnotation, 64)) { + // In `ParquetToSparkSchemaConverter`, we map parquet UINT64 to our Decimal(20, 0). + // For unsigned int64, it stores as plain signed int64 in Parquet when dictionary + // fallbacks. We read them as decimal values. + return; + } else if (isTimestampTypeMatched(logicalTypeAnnotation, TimeUnit.MICROS)) { + validateTimestampType(logicalTypeAnnotation, sparkType); + // TODO: use dateTimeRebaseMode from Spark side + return; + } else if (isTimestampTypeMatched(logicalTypeAnnotation, TimeUnit.MILLIS)) { + validateTimestampType(logicalTypeAnnotation, sparkType); + return; + } + break; + case INT96: + if (sparkType == TimestampNTZType$.MODULE$) { + convertErrorForTimestampNTZ(typeName.name()); + } else if (sparkType == DataTypes.TimestampType) { + return; + } + break; + case FLOAT: + if (sparkType == DataTypes.FloatType) return; + // In Comet we allow schema evolution from float to double, if + // `spark.comet.schemaEvolution.enabled` is enabled. + if (sparkType == DataTypes.DoubleType && allowTypePromotion) return; + break; + case DOUBLE: + if (sparkType == DataTypes.DoubleType) return; + break; + case BINARY: + if (sparkType == DataTypes.StringType + || sparkType == DataTypes.BinaryType + || canReadAsBinaryDecimal(descriptor, sparkType)) { + return; + } + break; + case FIXED_LEN_BYTE_ARRAY: + if (canReadAsIntDecimal(descriptor, sparkType) + || canReadAsLongDecimal(descriptor, sparkType) + || canReadAsBinaryDecimal(descriptor, sparkType) + || sparkType == DataTypes.BinaryType + // for uuid, since iceberg maps uuid to StringType + || sparkType == DataTypes.StringType) { + return; + } + break; + default: + break; + } + + throw new SchemaColumnConvertNotSupportedException( + Arrays.toString(descriptor.getPath()), + descriptor.getPrimitiveType().getPrimitiveTypeName().toString(), + sparkType.catalogString()); + } + + private static void validateTimestampType( + LogicalTypeAnnotation logicalTypeAnnotation, DataType sparkType) { + assert (logicalTypeAnnotation instanceof TimestampLogicalTypeAnnotation); + // Throw an exception if the Parquet type is TimestampLTZ and the Catalyst type is TimestampNTZ. + // This is to avoid mistakes in reading the timestamp values. + if (((TimestampLogicalTypeAnnotation) logicalTypeAnnotation).isAdjustedToUTC() + && sparkType == TimestampNTZType$.MODULE$) { + convertErrorForTimestampNTZ("int64 time(" + logicalTypeAnnotation + ")"); + } + } + + private static void convertErrorForTimestampNTZ(String parquetType) { + throw new RuntimeException( + "Unable to create Parquet converter for data type " + + TimestampNTZType$.MODULE$.json() + + " whose Parquet type is " + + parquetType); + } + + private static boolean canReadAsIntDecimal(ColumnDescriptor descriptor, DataType dt) { + if (!DecimalType.is32BitDecimalType(dt)) return false; + return isDecimalTypeMatched(descriptor, dt); + } + + private static boolean canReadAsLongDecimal(ColumnDescriptor descriptor, DataType dt) { + if (!DecimalType.is64BitDecimalType(dt)) return false; + return isDecimalTypeMatched(descriptor, dt); + } + + private static boolean canReadAsBinaryDecimal(ColumnDescriptor descriptor, DataType dt) { + if (!DecimalType.isByteArrayDecimalType(dt)) return false; + return isDecimalTypeMatched(descriptor, dt); + } + + private static boolean isLongDecimal(DataType dt) { + if (dt instanceof DecimalType) { + DecimalType d = (DecimalType) dt; + return d.precision() == 20 && d.scale() == 0; + } + return false; + } + + private static boolean isDecimalTypeMatched(ColumnDescriptor descriptor, DataType dt) { + DecimalType d = (DecimalType) dt; + LogicalTypeAnnotation typeAnnotation = descriptor.getPrimitiveType().getLogicalTypeAnnotation(); + if (typeAnnotation instanceof DecimalLogicalTypeAnnotation) { + DecimalLogicalTypeAnnotation decimalType = (DecimalLogicalTypeAnnotation) typeAnnotation; + // It's OK if the required decimal precision is larger than or equal to the physical decimal + // precision in the Parquet metadata, as long as the decimal scale is the same. + return decimalType.getPrecision() <= d.precision() && decimalType.getScale() == d.scale(); + } + return false; + } + + private static boolean isTimestampTypeMatched( + LogicalTypeAnnotation logicalTypeAnnotation, LogicalTypeAnnotation.TimeUnit unit) { + return logicalTypeAnnotation instanceof TimestampLogicalTypeAnnotation + && ((TimestampLogicalTypeAnnotation) logicalTypeAnnotation).getUnit() == unit; + } + + private static boolean isUnsignedIntTypeMatched( + LogicalTypeAnnotation logicalTypeAnnotation, int bitWidth) { + return logicalTypeAnnotation instanceof IntLogicalTypeAnnotation + && !((IntLogicalTypeAnnotation) logicalTypeAnnotation).isSigned() + && ((IntLogicalTypeAnnotation) logicalTypeAnnotation).getBitWidth() == bitWidth; + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/Utils.java b/common/src/main/java/org/apache/comet/parquet/Utils.java new file mode 100644 index 000000000..95ca06cda --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/Utils.java @@ -0,0 +1,238 @@ +/* + * 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.comet.parquet; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.spark.sql.types.*; + +public class Utils { + public static ColumnReader getColumnReader( + DataType type, + ColumnDescriptor descriptor, + int batchSize, + boolean useDecimal128, + boolean useLazyMaterialization) { + // TODO: support `useLegacyDateTimestamp` for Iceberg + return getColumnReader( + type, descriptor, batchSize, useDecimal128, useLazyMaterialization, true); + } + + public static ColumnReader getColumnReader( + DataType type, + ColumnDescriptor descriptor, + int batchSize, + boolean useDecimal128, + boolean useLazyMaterialization, + boolean useLegacyDateTimestamp) { + if (useLazyMaterialization && supportLazyMaterialization(type)) { + return new LazyColumnReader( + type, descriptor, batchSize, useDecimal128, useLegacyDateTimestamp); + } else { + return new ColumnReader(type, descriptor, batchSize, useDecimal128, useLegacyDateTimestamp); + } + } + + private static boolean supportLazyMaterialization(DataType type) { + return (type instanceof StringType || type instanceof BinaryType); + } + + /** + * Initialize the Comet native Parquet reader. + * + * @param descriptor the Parquet column descriptor for the column to be read + * @param readType the Spark read type used for type promotion. Null if promotion is not enabled. + * @param batchSize the batch size, i.e., maximum number of elements per record batch + * @param useDecimal128 whether to always represent decimals using 128 bits. If false, the native + * reader may represent decimals using 32 or 64 bits, depending on the precision. + * @param useLegacyDateTimestampOrNTZ whether to read dates/timestamps that were written in the + * legacy hybrid Julian + Gregorian calendar as it is. If false, throw exceptions instead. If + * the spark type is TimestampNTZ, this should be true. + */ + public static long initColumnReader( + ColumnDescriptor descriptor, + DataType readType, + int batchSize, + boolean useDecimal128, + boolean useLegacyDateTimestampOrNTZ) { + PrimitiveType primitiveType = descriptor.getPrimitiveType(); + int primitiveTypeId = getPhysicalTypeId(primitiveType.getPrimitiveTypeName()); + LogicalTypeAnnotation annotation = primitiveType.getLogicalTypeAnnotation(); + + // Process logical type information + + int bitWidth = -1; + boolean isSigned = false; + if (annotation instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { + LogicalTypeAnnotation.IntLogicalTypeAnnotation intAnnotation = + (LogicalTypeAnnotation.IntLogicalTypeAnnotation) annotation; + bitWidth = intAnnotation.getBitWidth(); + isSigned = intAnnotation.isSigned(); + } + + int precision, scale; + precision = scale = -1; + if (annotation instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { + LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalAnnotation = + (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) annotation; + precision = decimalAnnotation.getPrecision(); + scale = decimalAnnotation.getScale(); + } + + int tu = -1; + boolean isAdjustedUtc = false; + if (annotation instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { + LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampAnnotation = + (LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) annotation; + tu = getTimeUnitId(timestampAnnotation.getUnit()); + isAdjustedUtc = timestampAnnotation.isAdjustedToUTC(); + } + + TypePromotionInfo promotionInfo; + if (readType != null) { + promotionInfo = new TypePromotionInfo(readType); + } else { + // If type promotion is not enable, we'll just use the Parquet primitive type and precision. + promotionInfo = new TypePromotionInfo(primitiveTypeId, precision); + } + + return Native.initColumnReader( + primitiveTypeId, + getLogicalTypeId(annotation), + promotionInfo.physicalTypeId, + descriptor.getPath(), + descriptor.getMaxDefinitionLevel(), + descriptor.getMaxRepetitionLevel(), + bitWidth, + isSigned, + primitiveType.getTypeLength(), + precision, + promotionInfo.precision, + scale, + tu, + isAdjustedUtc, + batchSize, + useDecimal128, + useLegacyDateTimestampOrNTZ); + } + + static class TypePromotionInfo { + // The Parquet physical type ID converted from the Spark read schema, or the original Parquet + // physical type ID if type promotion is not enabled. + int physicalTypeId; + // Decimal precision from the Spark read schema, or -1 if it's not decimal type. + int precision; + + TypePromotionInfo(int physicalTypeId, int precision) { + this.physicalTypeId = physicalTypeId; + this.precision = precision; + } + + TypePromotionInfo(DataType sparkReadType) { + // Create a dummy `StructField` from the input Spark type. We don't care about + // field name, nullability and metadata. + StructField f = new StructField("f", sparkReadType, false, Metadata.empty()); + ColumnDescriptor descriptor = TypeUtil.convertToParquet(f); + PrimitiveType primitiveType = descriptor.getPrimitiveType(); + int physicalTypeId = getPhysicalTypeId(primitiveType.getPrimitiveTypeName()); + LogicalTypeAnnotation annotation = primitiveType.getLogicalTypeAnnotation(); + int precision = -1; + if (annotation instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { + LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalAnnotation = + (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) annotation; + precision = decimalAnnotation.getPrecision(); + } + this.physicalTypeId = physicalTypeId; + this.precision = precision; + } + } + + /** + * Maps the input Parquet physical type 'typeName' to an integer representing it. This is used for + * serialization between the Java and native side. + * + * @param typeName enum for the Parquet physical type + * @return an integer representing the input physical type + */ + static int getPhysicalTypeId(PrimitiveType.PrimitiveTypeName typeName) { + switch (typeName) { + case BOOLEAN: + return 0; + case INT32: + return 1; + case INT64: + return 2; + case INT96: + return 3; + case FLOAT: + return 4; + case DOUBLE: + return 5; + case BINARY: + return 6; + case FIXED_LEN_BYTE_ARRAY: + return 7; + } + throw new IllegalArgumentException("Invalid Parquet physical type: " + typeName); + } + + /** + * Maps the input Parquet logical type 'annotation' to an integer representing it. This is used + * for serialization between the Java and native side. + * + * @param annotation the Parquet logical type annotation + * @return an integer representing the input logical type + */ + static int getLogicalTypeId(LogicalTypeAnnotation annotation) { + if (annotation == null) { + return -1; // No logical type associated + } else if (annotation instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { + return 0; + } else if (annotation instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation) { + return 1; + } else if (annotation instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { + return 2; + } else if (annotation instanceof LogicalTypeAnnotation.DateLogicalTypeAnnotation) { + return 3; + } else if (annotation instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { + return 4; + } else if (annotation instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) { + return 5; + } else if (annotation instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { + return 6; + } + + throw new UnsupportedOperationException("Unsupported Parquet logical type " + annotation); + } + + static int getTimeUnitId(LogicalTypeAnnotation.TimeUnit tu) { + switch (tu) { + case MILLIS: + return 0; + case MICROS: + return 1; + case NANOS: + return 2; + default: + throw new UnsupportedOperationException("Unsupported TimeUnit " + tu); + } + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometDecodedVector.java b/common/src/main/java/org/apache/comet/vector/CometDecodedVector.java new file mode 100644 index 000000000..fec266bf1 --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometDecodedVector.java @@ -0,0 +1,106 @@ +/* + * 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.comet.vector; + +import org.apache.arrow.vector.BaseVariableWidthVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.spark.sql.comet.util.Utils; +import org.apache.spark.unsafe.Platform; + +/** A Comet vector whose elements are already decoded (i.e., materialized). */ +public abstract class CometDecodedVector extends CometVector { + /** + * The vector that stores all the values. For dictionary-backed vector, this is the vector of + * indices. + */ + protected final ValueVector valueVector; + + private boolean hasNull; + private int numNulls; + private int numValues; + private int validityByteCacheIndex = -1; + private byte validityByteCache; + + protected CometDecodedVector(ValueVector vector, Field valueField, boolean useDecimal128) { + super(Utils.fromArrowField(valueField), useDecimal128); + this.valueVector = vector; + this.numNulls = valueVector.getNullCount(); + this.numValues = valueVector.getValueCount(); + this.hasNull = numNulls != 0; + } + + @Override + ValueVector getValueVector() { + return valueVector; + } + + @Override + public void setNumNulls(int numNulls) { + // We don't need to update null count in 'valueVector' since 'ValueVector.getNullCount' will + // re-compute the null count from validity buffer. + this.numNulls = numNulls; + this.hasNull = numNulls != 0; + this.validityByteCacheIndex = -1; + } + + @Override + public void setNumValues(int numValues) { + this.numValues = numValues; + if (valueVector instanceof BaseVariableWidthVector) { + BaseVariableWidthVector bv = (BaseVariableWidthVector) valueVector; + // In case `lastSet` is smaller than `numValues`, `setValueCount` will set all the offsets + // within `[lastSet + 1, numValues)` to be empty, which is incorrect in our case. + // + // For instance, this can happen if one first call `setNumValues` with input 100, and then + // again `setNumValues` with 200. The first call will set `lastSet` to 99, while the second + // call will set all strings between indices `[100, 200)` to be empty. + bv.setLastSet(numValues); + } + valueVector.setValueCount(numValues); + } + + public int numValues() { + return numValues; + } + + @Override + public boolean hasNull() { + return hasNull; + } + + @Override + public int numNulls() { + return numNulls; + } + + @Override + public boolean isNullAt(int rowId) { + if (!hasNull) return false; + + int byteIndex = rowId >> 3; + if (byteIndex != validityByteCacheIndex) { + long validityBufferAddress = valueVector.getValidityBuffer().memoryAddress(); + validityByteCache = Platform.getByte(null, validityBufferAddress + byteIndex); + validityByteCacheIndex = byteIndex; + } + return ((validityByteCache >> (rowId & 7)) & 1) == 0; + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometDelegateVector.java b/common/src/main/java/org/apache/comet/vector/CometDelegateVector.java new file mode 100644 index 000000000..165af5631 --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometDelegateVector.java @@ -0,0 +1,169 @@ +/* + * 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.comet.vector; + +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.dictionary.DictionaryProvider; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.types.UTF8String; + +/** A special Comet vector that just delegate all calls */ +public class CometDelegateVector extends CometVector { + protected CometVector delegate; + + public CometDelegateVector(DataType dataType) { + this(dataType, null, false); + } + + public CometDelegateVector(DataType dataType, boolean useDecimal128) { + this(dataType, null, useDecimal128); + } + + public CometDelegateVector(DataType dataType, CometVector delegate, boolean useDecimal128) { + super(dataType, useDecimal128); + if (delegate instanceof CometDelegateVector) { + throw new IllegalArgumentException("cannot have nested delegation"); + } + this.delegate = delegate; + } + + public void setDelegate(CometVector delegate) { + this.delegate = delegate; + } + + @Override + public void setNumNulls(int numNulls) { + delegate.setNumNulls(numNulls); + } + + @Override + public void setNumValues(int numValues) { + delegate.setNumValues(numValues); + } + + @Override + public int numValues() { + return delegate.numValues(); + } + + @Override + public boolean hasNull() { + return delegate.hasNull(); + } + + @Override + public int numNulls() { + return delegate.numNulls(); + } + + @Override + public boolean isNullAt(int rowId) { + return delegate.isNullAt(rowId); + } + + @Override + public boolean getBoolean(int rowId) { + return delegate.getBoolean(rowId); + } + + @Override + public byte getByte(int rowId) { + return delegate.getByte(rowId); + } + + @Override + public short getShort(int rowId) { + return delegate.getShort(rowId); + } + + @Override + public int getInt(int rowId) { + return delegate.getInt(rowId); + } + + @Override + public long getLong(int rowId) { + return delegate.getLong(rowId); + } + + @Override + public float getFloat(int rowId) { + return delegate.getFloat(rowId); + } + + @Override + public double getDouble(int rowId) { + return delegate.getDouble(rowId); + } + + @Override + public Decimal getDecimal(int i, int precision, int scale) { + return delegate.getDecimal(i, precision, scale); + } + + @Override + byte[] getBinaryDecimal(int i) { + return delegate.getBinaryDecimal(i); + } + + @Override + public UTF8String getUTF8String(int rowId) { + return delegate.getUTF8String(rowId); + } + + @Override + public byte[] getBinary(int rowId) { + return delegate.getBinary(rowId); + } + + @Override + public ColumnarArray getArray(int i) { + return delegate.getArray(i); + } + + @Override + public ColumnarMap getMap(int i) { + return delegate.getMap(i); + } + + @Override + public ColumnVector getChild(int i) { + return delegate.getChild(i); + } + + @Override + ValueVector getValueVector() { + return delegate.getValueVector(); + } + + @Override + public CometVector slice(int offset, int length) { + return delegate.slice(offset, length); + } + + @Override + DictionaryProvider getDictionaryProvider() { + return delegate.getDictionaryProvider(); + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometDictionary.java b/common/src/main/java/org/apache/comet/vector/CometDictionary.java new file mode 100644 index 000000000..8c7046585 --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometDictionary.java @@ -0,0 +1,178 @@ +/* + * 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.comet.vector; + +import org.apache.arrow.vector.ValueVector; +import org.apache.spark.unsafe.types.UTF8String; + +/** A dictionary which maps indices (integers) to values. */ +public class CometDictionary implements AutoCloseable { + private static final int DECIMAL_BYTE_WIDTH = 16; + + private final CometPlainVector values; + private final int numValues; + + /** Decoded dictionary values. Only one of the following is set. */ + private byte[] bytes; + + private short[] shorts; + private int[] ints; + private long[] longs; + private float[] floats; + private double[] doubles; + private boolean[] booleans; + private ByteArrayWrapper[] binaries; + private UTF8String[] strings; + + public CometDictionary(CometPlainVector values) { + this.values = values; + this.numValues = values.numValues(); + initialize(); + } + + public ValueVector getValueVector() { + return values.getValueVector(); + } + + public boolean decodeToBoolean(int index) { + return booleans[index]; + } + + public byte decodeToByte(int index) { + return bytes[index]; + } + + public short decodeToShort(int index) { + return shorts[index]; + } + + public int decodeToInt(int index) { + return ints[index]; + } + + public long decodeToLong(int index) { + return longs[index]; + } + + public float decodeToFloat(int index) { + return floats[index]; + } + + public double decodeToDouble(int index) { + return doubles[index]; + } + + public byte[] decodeToBinary(int index) { + return binaries[index].bytes; + } + + public UTF8String decodeToUTF8String(int index) { + return strings[index]; + } + + @Override + public void close() { + values.close(); + } + + private void initialize() { + switch (values.getValueVector().getMinorType()) { + case BIT: + booleans = new boolean[numValues]; + for (int i = 0; i < numValues; i++) { + booleans[i] = values.getBoolean(i); + } + break; + case TINYINT: + bytes = new byte[numValues]; + for (int i = 0; i < numValues; i++) { + bytes[i] = values.getByte(i); + } + break; + case SMALLINT: + shorts = new short[numValues]; + for (int i = 0; i < numValues; i++) { + shorts[i] = values.getShort(i); + } + break; + case INT: + case DATEDAY: + ints = new int[numValues]; + for (int i = 0; i < numValues; i++) { + ints[i] = values.getInt(i); + } + break; + case BIGINT: + case TIMESTAMPMICRO: + case TIMESTAMPMICROTZ: + longs = new long[numValues]; + for (int i = 0; i < numValues; i++) { + longs[i] = values.getLong(i); + } + break; + case FLOAT4: + floats = new float[numValues]; + for (int i = 0; i < numValues; i++) { + floats[i] = values.getFloat(i); + } + break; + case FLOAT8: + doubles = new double[numValues]; + for (int i = 0; i < numValues; i++) { + doubles[i] = values.getDouble(i); + } + break; + case VARBINARY: + case FIXEDSIZEBINARY: + binaries = new ByteArrayWrapper[numValues]; + for (int i = 0; i < numValues; i++) { + binaries[i] = new ByteArrayWrapper(values.getBinary(i)); + } + break; + case VARCHAR: + strings = new UTF8String[numValues]; + for (int i = 0; i < numValues; i++) { + strings[i] = values.getUTF8String(i); + } + break; + case DECIMAL: + binaries = new ByteArrayWrapper[numValues]; + for (int i = 0; i < numValues; i++) { + // Need copying here since we re-use byte array for decimal + byte[] bytes = values.getBinaryDecimal(i); + byte[] copy = new byte[DECIMAL_BYTE_WIDTH]; + System.arraycopy(bytes, 0, copy, 0, DECIMAL_BYTE_WIDTH); + binaries[i] = new ByteArrayWrapper(copy); + } + break; + default: + throw new IllegalArgumentException( + "Invalid Arrow minor type: " + values.getValueVector().getMinorType()); + } + } + + private static class ByteArrayWrapper { + private final byte[] bytes; + + ByteArrayWrapper(byte[] bytes) { + this.bytes = bytes; + } + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java b/common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java new file mode 100644 index 000000000..225fcfc43 --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java @@ -0,0 +1,135 @@ +/* + * 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.comet.vector; + +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.dictionary.DictionaryProvider; +import org.apache.arrow.vector.util.TransferPair; +import org.apache.parquet.Preconditions; +import org.apache.spark.unsafe.types.UTF8String; + +/** A column vector whose elements are dictionary-encoded. */ +public class CometDictionaryVector extends CometDecodedVector { + public final CometPlainVector indices; + public final CometDictionary values; + public final DictionaryProvider provider; + + /** Whether this vector is an alias sliced from another vector. */ + private final boolean isAlias; + + public CometDictionaryVector( + CometPlainVector indices, + CometDictionary values, + DictionaryProvider provider, + boolean useDecimal128) { + this(indices, values, provider, useDecimal128, false); + } + + public CometDictionaryVector( + CometPlainVector indices, + CometDictionary values, + DictionaryProvider provider, + boolean useDecimal128, + boolean isAlias) { + super(indices.valueVector, values.getValueVector().getField(), useDecimal128); + Preconditions.checkArgument( + indices.valueVector instanceof IntVector, "'indices' should be a IntVector"); + this.values = values; + this.indices = indices; + this.provider = provider; + this.isAlias = isAlias; + } + + @Override + DictionaryProvider getDictionaryProvider() { + return this.provider; + } + + @Override + public void close() { + super.close(); + // Only close the values vector if this is not a sliced vector. + if (!isAlias) { + values.close(); + } + } + + @Override + public boolean getBoolean(int i) { + return values.decodeToBoolean(indices.getInt(i)); + } + + @Override + public byte getByte(int i) { + return values.decodeToByte(indices.getInt(i)); + } + + @Override + public short getShort(int i) { + return values.decodeToShort(indices.getInt(i)); + } + + @Override + public int getInt(int i) { + return values.decodeToInt(indices.getInt(i)); + } + + @Override + public long getLong(int i) { + return values.decodeToLong(indices.getInt(i)); + } + + @Override + public float getFloat(int i) { + return values.decodeToFloat(indices.getInt(i)); + } + + @Override + public double getDouble(int i) { + return values.decodeToDouble(indices.getInt(i)); + } + + @Override + public UTF8String getUTF8String(int i) { + return values.decodeToUTF8String(indices.getInt(i)); + } + + @Override + public byte[] getBinary(int i) { + return values.decodeToBinary(indices.getInt(i)); + } + + @Override + byte[] getBinaryDecimal(int i) { + return values.decodeToBinary(indices.getInt(i)); + } + + @Override + public CometVector slice(int offset, int length) { + TransferPair tp = indices.valueVector.getTransferPair(indices.valueVector.getAllocator()); + tp.splitAndTransfer(offset, length); + CometPlainVector sliced = new CometPlainVector(tp.getTo(), useDecimal128); + + // Set the alias flag to true so that the sliced vector will not close the dictionary vector. + // Otherwise, if the dictionary is closed, the sliced vector will not be able to access the + // dictionary. + return new CometDictionaryVector(sliced, values, provider, useDecimal128, true); + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometLazyVector.java b/common/src/main/java/org/apache/comet/vector/CometLazyVector.java new file mode 100644 index 000000000..17b8d7e71 --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometLazyVector.java @@ -0,0 +1,82 @@ +/* + * 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.comet.vector; + +import org.apache.arrow.vector.ValueVector; +import org.apache.spark.sql.types.DataType; + +import org.apache.comet.parquet.LazyColumnReader; + +public class CometLazyVector extends CometDelegateVector { + private final LazyColumnReader columnReader; + + public CometLazyVector(DataType type, LazyColumnReader columnReader, boolean useDecimal128) { + super(type, useDecimal128); + this.columnReader = columnReader; + } + + public CometDecodedVector getDecodedVector() { + return (CometDecodedVector) delegate; + } + + @Override + public ValueVector getValueVector() { + columnReader.readAllBatch(); + setDelegate(columnReader.loadVector()); + return super.getValueVector(); + } + + @Override + public void setNumNulls(int numNulls) { + throw new UnsupportedOperationException("CometLazyVector doesn't support 'setNumNulls'"); + } + + @Override + public void setNumValues(int numValues) { + throw new UnsupportedOperationException("CometLazyVector doesn't support 'setNumValues'"); + } + + @Override + public void close() { + // Do nothing. 'vector' is closed by 'columnReader' which owns it. + } + + @Override + public boolean hasNull() { + columnReader.readAllBatch(); + setDelegate(columnReader.loadVector()); + return super.hasNull(); + } + + @Override + public int numNulls() { + columnReader.readAllBatch(); + setDelegate(columnReader.loadVector()); + return super.numNulls(); + } + + @Override + public boolean isNullAt(int rowId) { + if (columnReader.materializeUpToIfNecessary(rowId)) { + setDelegate(columnReader.loadVector()); + } + return super.isNullAt(rowId); + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometListVector.java b/common/src/main/java/org/apache/comet/vector/CometListVector.java new file mode 100644 index 000000000..1c8f3e658 --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometListVector.java @@ -0,0 +1,57 @@ +/* + * 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.comet.vector; + +import org.apache.arrow.vector.*; +import org.apache.arrow.vector.complex.ListVector; +import org.apache.arrow.vector.util.TransferPair; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; + +/** A Comet column vector for list type. */ +public class CometListVector extends CometDecodedVector { + final ListVector listVector; + final ValueVector dataVector; + final ColumnVector dataColumnVector; + + public CometListVector(ValueVector vector, boolean useDecimal128) { + super(vector, vector.getField(), useDecimal128); + + this.listVector = ((ListVector) vector); + this.dataVector = listVector.getDataVector(); + this.dataColumnVector = getVector(dataVector, useDecimal128); + } + + @Override + public ColumnarArray getArray(int i) { + int start = listVector.getOffsetBuffer().getInt(i * ListVector.OFFSET_WIDTH); + int end = listVector.getOffsetBuffer().getInt((i + 1) * ListVector.OFFSET_WIDTH); + + return new ColumnarArray(dataColumnVector, start, end - start); + } + + @Override + public CometVector slice(int offset, int length) { + TransferPair tp = this.valueVector.getTransferPair(this.valueVector.getAllocator()); + tp.splitAndTransfer(offset, length); + + return new CometListVector(tp.getTo(), useDecimal128); + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometPlainVector.java b/common/src/main/java/org/apache/comet/vector/CometPlainVector.java new file mode 100644 index 000000000..a7373590a --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometPlainVector.java @@ -0,0 +1,172 @@ +/* + * 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.comet.vector; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.UUID; + +import org.apache.arrow.c.CDataDictionaryProvider; +import org.apache.arrow.vector.*; +import org.apache.arrow.vector.util.TransferPair; +import org.apache.parquet.Preconditions; +import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.types.UTF8String; + +/** A column vector whose elements are plainly decoded. */ +public class CometPlainVector extends CometDecodedVector { + private final long valueBufferAddress; + private final boolean isBaseFixedWidthVector; + + private byte booleanByteCache; + private int booleanByteCacheIndex = -1; + + public CometPlainVector(ValueVector vector, boolean useDecimal128) { + super(vector, vector.getField(), useDecimal128); + // NullType doesn't have data buffer. + if (vector instanceof NullVector) { + this.valueBufferAddress = -1; + } else { + this.valueBufferAddress = vector.getDataBuffer().memoryAddress(); + } + + isBaseFixedWidthVector = valueVector instanceof BaseFixedWidthVector; + } + + @Override + public void setNumNulls(int numNulls) { + super.setNumNulls(numNulls); + this.booleanByteCacheIndex = -1; + } + + @Override + public boolean getBoolean(int rowId) { + int byteIndex = rowId >> 3; + if (byteIndex != booleanByteCacheIndex) { + booleanByteCache = getByte(byteIndex); + booleanByteCacheIndex = byteIndex; + } + return ((booleanByteCache >> (rowId & 7)) & 1) == 1; + } + + @Override + public byte getByte(int rowId) { + return Platform.getByte(null, valueBufferAddress + rowId); + } + + @Override + public short getShort(int rowId) { + return Platform.getShort(null, valueBufferAddress + rowId * 2L); + } + + @Override + public int getInt(int rowId) { + return Platform.getInt(null, valueBufferAddress + rowId * 4L); + } + + @Override + public long getLong(int rowId) { + return Platform.getLong(null, valueBufferAddress + rowId * 8L); + } + + @Override + public float getFloat(int rowId) { + return Platform.getFloat(null, valueBufferAddress + rowId * 4L); + } + + @Override + public double getDouble(int rowId) { + return Platform.getDouble(null, valueBufferAddress + rowId * 8L); + } + + @Override + public UTF8String getUTF8String(int rowId) { + if (!isBaseFixedWidthVector) { + BaseVariableWidthVector varWidthVector = (BaseVariableWidthVector) valueVector; + long offsetBufferAddress = varWidthVector.getOffsetBuffer().memoryAddress(); + int offset = Platform.getInt(null, offsetBufferAddress + rowId * 4L); + int length = Platform.getInt(null, offsetBufferAddress + (rowId + 1L) * 4L) - offset; + return UTF8String.fromAddress(null, valueBufferAddress + offset, length); + } else { + // Iceberg maps UUID to StringType. + // The data type here must be UUID because the only FLBA -> String mapping we have is UUID. + BaseFixedWidthVector fixedWidthVector = (BaseFixedWidthVector) valueVector; + int length = fixedWidthVector.getTypeWidth(); + int offset = rowId * length; + byte[] result = new byte[length]; + Platform.copyMemory( + null, valueBufferAddress + offset, result, Platform.BYTE_ARRAY_OFFSET, length); + return UTF8String.fromString(convertToUuid(result).toString()); + } + } + + @Override + public byte[] getBinary(int rowId) { + int offset; + int length; + if (valueVector instanceof BaseVariableWidthVector) { + BaseVariableWidthVector varWidthVector = (BaseVariableWidthVector) valueVector; + long offsetBufferAddress = varWidthVector.getOffsetBuffer().memoryAddress(); + offset = Platform.getInt(null, offsetBufferAddress + rowId * 4L); + length = Platform.getInt(null, offsetBufferAddress + (rowId + 1L) * 4L) - offset; + } else if (valueVector instanceof BaseFixedWidthVector) { + BaseFixedWidthVector fixedWidthVector = (BaseFixedWidthVector) valueVector; + length = fixedWidthVector.getTypeWidth(); + offset = rowId * length; + } else { + throw new RuntimeException("Unsupported binary vector type: " + valueVector.getName()); + } + byte[] result = new byte[length]; + Platform.copyMemory( + null, valueBufferAddress + offset, result, Platform.BYTE_ARRAY_OFFSET, length); + return result; + } + + @Override + CDataDictionaryProvider getDictionaryProvider() { + return null; + } + + @Override + public boolean isNullAt(int rowId) { + if (this.valueBufferAddress == -1) { + return true; + } else { + return super.isNullAt(rowId); + } + } + + @Override + public CometVector slice(int offset, int length) { + TransferPair tp = this.valueVector.getTransferPair(this.valueVector.getAllocator()); + tp.splitAndTransfer(offset, length); + + return new CometPlainVector(tp.getTo(), useDecimal128); + } + + private static UUID convertToUuid(byte[] buf) { + Preconditions.checkArgument(buf.length == 16, "UUID require 16 bytes"); + ByteBuffer bb = ByteBuffer.wrap(buf); + bb.order(ByteOrder.BIG_ENDIAN); + long mostSigBits = bb.getLong(); + long leastSigBits = bb.getLong(); + return new UUID(mostSigBits, leastSigBits); + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometStructVector.java b/common/src/main/java/org/apache/comet/vector/CometStructVector.java new file mode 100644 index 000000000..52dcd4326 --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometStructVector.java @@ -0,0 +1,61 @@ +/* + * 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.comet.vector; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.arrow.vector.*; +import org.apache.arrow.vector.complex.StructVector; +import org.apache.arrow.vector.util.TransferPair; +import org.apache.spark.sql.vectorized.ColumnVector; + +/** A Comet column vector for struct type. */ +public class CometStructVector extends CometDecodedVector { + final List children; + + public CometStructVector(ValueVector vector, boolean useDecimal128) { + super(vector, vector.getField(), useDecimal128); + + StructVector structVector = ((StructVector) vector); + + int size = structVector.size(); + List children = new ArrayList<>(); + + for (int i = 0; i < size; ++i) { + ValueVector value = structVector.getVectorById(i); + children.add(getVector(value, useDecimal128)); + } + this.children = children; + } + + @Override + public ColumnVector getChild(int i) { + return children.get(i); + } + + @Override + public CometVector slice(int offset, int length) { + TransferPair tp = this.valueVector.getTransferPair(this.valueVector.getAllocator()); + tp.splitAndTransfer(offset, length); + + return new CometStructVector(tp.getTo(), useDecimal128); + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometVector.java b/common/src/main/java/org/apache/comet/vector/CometVector.java new file mode 100644 index 000000000..744f24395 --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometVector.java @@ -0,0 +1,232 @@ +/* + * 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.comet.vector; + +import java.math.BigDecimal; +import java.math.BigInteger; + +import org.apache.arrow.vector.FixedWidthVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.complex.ListVector; +import org.apache.arrow.vector.complex.StructVector; +import org.apache.arrow.vector.dictionary.Dictionary; +import org.apache.arrow.vector.dictionary.DictionaryProvider; +import org.apache.arrow.vector.types.pojo.DictionaryEncoding; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.types.UTF8String; + +/** Base class for all Comet column vector implementations. */ +public abstract class CometVector extends ColumnVector { + private static final int DECIMAL_BYTE_WIDTH = 16; + private final byte[] DECIMAL_BYTES = new byte[DECIMAL_BYTE_WIDTH]; + protected final boolean useDecimal128; + + protected CometVector(DataType type, boolean useDecimal128) { + super(type); + this.useDecimal128 = useDecimal128; + } + + /** + * Sets the number of nulls in this vector to be 'numNulls'. This is used when the vector is + * reused across batches. + */ + public abstract void setNumNulls(int numNulls); + + /** + * Sets the number of values (including both nulls and non-nulls) in this vector to be + * 'numValues'. This is used when the vector is reused across batches. + */ + public abstract void setNumValues(int numValues); + + /** Returns the number of values in this vector. */ + public abstract int numValues(); + + /** Whether the elements of this vector are of fixed length. */ + public boolean isFixedLength() { + return getValueVector() instanceof FixedWidthVector; + } + + @Override + public Decimal getDecimal(int i, int precision, int scale) { + if (!useDecimal128 && precision <= Decimal.MAX_INT_DIGITS() && type instanceof IntegerType) { + return Decimal.createUnsafe(getInt(i), precision, scale); + } else if (!useDecimal128 && precision <= Decimal.MAX_LONG_DIGITS()) { + return Decimal.createUnsafe(getLong(i), precision, scale); + } else { + byte[] bytes = getBinaryDecimal(i); + BigInteger bigInteger = new BigInteger(bytes); + BigDecimal javaDecimal = new BigDecimal(bigInteger, scale); + try { + return Decimal.apply(javaDecimal, precision, scale); + } catch (ArithmeticException e) { + throw new ArithmeticException( + "Cannot convert " + + javaDecimal + + " (bytes: " + + bytes + + ", integer: " + + bigInteger + + ") to decimal with precision: " + + precision + + " and scale: " + + scale); + } + } + } + + /** Reads a 16-byte byte array which are encoded big-endian for decimal128. */ + byte[] getBinaryDecimal(int i) { + long valueBufferAddress = getValueVector().getDataBuffer().memoryAddress(); + Platform.copyMemory( + null, + valueBufferAddress + (long) i * DECIMAL_BYTE_WIDTH, + DECIMAL_BYTES, + Platform.BYTE_ARRAY_OFFSET, + DECIMAL_BYTE_WIDTH); + // Decimal is stored little-endian in Arrow, so we need to reverse the bytes here + for (int j = 0, k = DECIMAL_BYTE_WIDTH - 1; j < DECIMAL_BYTE_WIDTH / 2; j++, k--) { + byte tmp = DECIMAL_BYTES[j]; + DECIMAL_BYTES[j] = DECIMAL_BYTES[k]; + DECIMAL_BYTES[k] = tmp; + } + return DECIMAL_BYTES; + } + + @Override + public boolean getBoolean(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public byte getByte(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public short getShort(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public int getInt(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public long getLong(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public float getFloat(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public double getDouble(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public UTF8String getUTF8String(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public byte[] getBinary(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public ColumnarArray getArray(int i) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public ColumnarMap getMap(int i) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public ColumnVector getChild(int i) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public void close() { + getValueVector().close(); + } + + DictionaryProvider getDictionaryProvider() { + throw new UnsupportedOperationException("Not implemented"); + } + + abstract ValueVector getValueVector(); + + /** + * Returns a zero-copying new vector that contains the values from [offset, offset + length). + * + * @param offset the offset of the new vector + * @param length the length of the new vector + * @return the new vector + */ + public abstract CometVector slice(int offset, int length); + + /** + * Returns a corresponding `CometVector` implementation based on the given Arrow `ValueVector`. + * + * @param vector Arrow `ValueVector` + * @param useDecimal128 Whether to use Decimal128 for decimal column + * @return `CometVector` implementation + */ + protected static CometVector getVector( + ValueVector vector, boolean useDecimal128, DictionaryProvider dictionaryProvider) { + if (vector instanceof StructVector) { + return new CometStructVector(vector, useDecimal128); + } else if (vector instanceof ListVector) { + return new CometListVector(vector, useDecimal128); + } else { + DictionaryEncoding dictionaryEncoding = vector.getField().getDictionary(); + CometPlainVector cometVector = new CometPlainVector(vector, useDecimal128); + + if (dictionaryEncoding == null) { + return cometVector; + } else { + Dictionary dictionary = dictionaryProvider.lookup(dictionaryEncoding.getId()); + CometPlainVector dictionaryVector = + new CometPlainVector(dictionary.getVector(), useDecimal128); + CometDictionary cometDictionary = new CometDictionary(dictionaryVector); + + return new CometDictionaryVector( + cometVector, cometDictionary, dictionaryProvider, useDecimal128); + } + } + } + + protected static CometVector getVector(ValueVector vector, boolean useDecimal128) { + return getVector(vector, useDecimal128, null); + } +} diff --git a/common/src/main/resources/log4j2.properties b/common/src/main/resources/log4j2.properties new file mode 100644 index 000000000..04cdf8533 --- /dev/null +++ b/common/src/main/resources/log4j2.properties @@ -0,0 +1,40 @@ +# 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. + +# Set everything to be logged to the file target/unit-tests.log +rootLogger.level = info +rootLogger.appenderRef.file.ref = ${sys:test.appender:-File} + +appender.file.type = File +appender.file.name = File +appender.file.fileName = target/unit-tests.log +appender.file.layout.type = PatternLayout +appender.file.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Tests that launch java subprocesses can set the "test.appender" system property to +# "console" to avoid having the child process's logs overwrite the unit test's +# log file. +appender.console.type = Console +appender.console.name = console +appender.console.target = SYSTEM_ERR +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %t: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +logger.jetty.name = org.sparkproject.jetty +logger.jetty.level = warn + diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala new file mode 100644 index 000000000..7f83d9296 --- /dev/null +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -0,0 +1,454 @@ +/* + * 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.comet + +import java.util.concurrent.TimeUnit + +import org.apache.spark.network.util.ByteUnit +import org.apache.spark.network.util.JavaUtils +import org.apache.spark.sql.comet.util.Utils +import org.apache.spark.sql.internal.SQLConf + +/** + * Configurations for a Comet application. Mostly inspired by [[SQLConf]] in Spark. + * + * To get the value of a Comet config key from a [[SQLConf]], you can do the following: + * + * {{{ + * CometConf.COMET_ENABLED.get + * }}} + * + * which retrieves the config value from the thread-local [[SQLConf]] object. Alternatively, you + * can also explicitly pass a [[SQLConf]] object to the `get` method. + */ +object CometConf { + def conf(key: String): ConfigBuilder = ConfigBuilder(key) + + val COMET_EXEC_CONFIG_PREFIX = "spark.comet.exec"; + + val COMET_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.enabled") + .doc( + "Whether to enable Comet extension for Spark. When this is turned on, Spark will use " + + "Comet to read Parquet data source. Note that to enable native vectorized execution, " + + "both this config and 'spark.comet.exec.enabled' need to be enabled. By default, this " + + "config is the value of the env var `ENABLE_COMET` if set, or true otherwise.") + .booleanConf + .createWithDefault(sys.env.getOrElse("ENABLE_COMET", "true").toBoolean) + + val COMET_SCAN_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.scan.enabled") + .doc( + "Whether to enable Comet scan. When this is turned on, Spark will use Comet to read " + + "Parquet data source. Note that to enable native vectorized execution, both this " + + "config and 'spark.comet.exec.enabled' need to be enabled. By default, this config " + + "is true.") + .booleanConf + .createWithDefault(true) + + val COMET_EXEC_ENABLED: ConfigEntry[Boolean] = conf(s"$COMET_EXEC_CONFIG_PREFIX.enabled") + .doc( + "Whether to enable Comet native vectorized execution for Spark. This controls whether " + + "Spark should convert operators into their Comet counterparts and execute them in " + + "native space. Note: each operator is associated with a separate config in the " + + "format of 'spark.comet.exec..enabled' at the moment, and both the " + + "config and this need to be turned on, in order for the operator to be executed in " + + "native. By default, this config is false.") + .booleanConf + .createWithDefault(false) + + val COMET_MEMORY_OVERHEAD: OptionalConfigEntry[Long] = conf("spark.comet.memoryOverhead") + .doc( + "The amount of additional memory to be allocated per executor process for Comet, in MiB. " + + "This config is optional. If this is not specified, it will be set to " + + "`spark.comet.memory.overhead.factor` * `spark.executor.memory`. " + + "This is memory that accounts for things like Comet native execution, etc.") + .bytesConf(ByteUnit.MiB) + .createOptional + + val COMET_MEMORY_OVERHEAD_FACTOR: ConfigEntry[Double] = conf( + "spark.comet.memory.overhead.factor") + .doc( + "Fraction of executor memory to be allocated as additional non-heap memory per executor " + + "process for Comet. Default value is 0.2.") + .doubleConf + .checkValue( + factor => factor > 0, + "Ensure that Comet memory overhead factor is a double greater than 0") + .createWithDefault(0.2) + + val COMET_MEMORY_OVERHEAD_MIN_MIB: ConfigEntry[Long] = conf("spark.comet.memory.overhead.min") + .doc("Minimum amount of additional memory to be allocated per executor process for Comet, " + + "in MiB.") + .bytesConf(ByteUnit.MiB) + .checkValue( + _ >= 0, + "Ensure that Comet memory overhead min is a long greater than or equal to 0") + .createWithDefault(384) + + val COMET_EXEC_ALL_OPERATOR_ENABLED: ConfigEntry[Boolean] = conf( + s"$COMET_EXEC_CONFIG_PREFIX.all.enabled") + .doc( + "Whether to enable all Comet operators. By default, this config is false. Note that " + + "this config precedes all separate config 'spark.comet.exec..enabled'. " + + "That being said, if this config is enabled, separate configs are ignored.") + .booleanConf + .createWithDefault(false) + + val COMET_EXEC_ALL_EXPR_ENABLED: ConfigEntry[Boolean] = + conf(s"$COMET_EXEC_CONFIG_PREFIX.all.expr.enabled") + .doc( + "Whether to enable all Comet exprs. By default, this config is false. Note that " + + "this config precedes all separate config 'spark.comet.exec..enabled'. " + + "That being said, if this config is enabled, separate configs are ignored.") + .booleanConf + .createWithDefault(false) + + val COMET_DEBUG_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.debug.enabled") + .doc( + "Whether to enable debug mode for Comet. By default, this config is false. " + + "When enabled, Comet will do additional checks for debugging purpose. For example, " + + "validating array when importing arrays from JVM at native side. Note that these " + + "checks may be expensive in performance and should only be enabled for debugging " + + "purpose.") + .booleanConf + .createWithDefault(false) + + val COMET_BATCH_SIZE: ConfigEntry[Int] = conf("spark.comet.batchSize") + .doc("The columnar batch size, i.e., the maximum number of rows that a batch can contain.") + .intConf + .createWithDefault(8192) + + val COMET_EXEC_MEMORY_FRACTION: ConfigEntry[Double] = conf("spark.comet.exec.memoryFraction") + .doc( + "The fraction of memory from Comet memory overhead that the native memory " + + "manager can use for execution. The purpose of this config is to set aside memory for " + + "untracked data structures, as well as imprecise size estimation during memory " + + "acquisition. Default value is 0.7.") + .doubleConf + .createWithDefault(0.7) + + val COMET_PARQUET_ENABLE_DIRECT_BUFFER: ConfigEntry[Boolean] = conf( + "spark.comet.parquet.enable.directBuffer") + .doc("Whether to use Java direct byte buffer when reading Parquet. By default, this is false") + .booleanConf + .createWithDefault(false) + + val COMET_SCAN_PREFETCH_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.scan.preFetch.enabled") + .doc("Whether to enable pre-fetching feature of CometScan. By default is disabled.") + .booleanConf + .createWithDefault(false) + + val COMET_SCAN_PREFETCH_THREAD_NUM: ConfigEntry[Int] = + conf("spark.comet.scan.preFetch.threadNum") + .doc( + "The number of threads running pre-fetching for CometScan. Effective if " + + s"${COMET_SCAN_PREFETCH_ENABLED.key} is enabled. By default it is 2. Note that more " + + "pre-fetching threads means more memory requirement to store pre-fetched row groups.") + .intConf + .createWithDefault(2) + + val COMET_NATIVE_LOAD_REQUIRED: ConfigEntry[Boolean] = conf("spark.comet.nativeLoadRequired") + .doc( + "Whether to require Comet native library to load successfully when Comet is enabled. " + + "If not, Comet will silently fallback to Spark when it fails to load the native lib. " + + "Otherwise, an error will be thrown and the Spark job will be aborted.") + .booleanConf + .createWithDefault(false) + + val COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP: ConfigEntry[Boolean] = + conf("spark.comet.exceptionOnDatetimeRebase") + .doc("Whether to throw exception when seeing dates/timestamps from the legacy hybrid " + + "(Julian + Gregorian) calendar. Since Spark 3, dates/timestamps were written according " + + "to the Proleptic Gregorian calendar. When this is true, Comet will " + + "throw exceptions when seeing these dates/timestamps that were written by Spark version " + + "before 3.0. If this is false, these dates/timestamps will be read as if they were " + + "written to the Proleptic Gregorian calendar and will not be rebased.") + .booleanConf + .createWithDefault(false) + + val COMET_USE_DECIMAL_128: ConfigEntry[Boolean] = conf("spark.comet.use.decimal128") + .internal() + .doc("If true, Comet will always use 128 bits to represent a decimal value, regardless of " + + "its precision. If false, Comet will use 32, 64 and 128 bits respectively depending on " + + "the precision. N.B. this is NOT a user-facing config but should be inferred and set by " + + "Comet itself.") + .booleanConf + .createWithDefault(false) + + val COMET_USE_LAZY_MATERIALIZATION: ConfigEntry[Boolean] = conf( + "spark.comet.use.lazyMaterialization") + .internal() + .doc( + "Whether to enable lazy materialization for Comet. When this is turned on, Comet will " + + "read Parquet data source lazily for string and binary columns. For filter operations, " + + "lazy materialization will improve read performance by skipping unused pages.") + .booleanConf + .createWithDefault(true) + + val COMET_SCHEMA_EVOLUTION_ENABLED: ConfigEntry[Boolean] = conf( + "spark.comet.schemaEvolution.enabled") + .internal() + .doc( + "Whether to enable schema evolution in Comet. For instance, promoting a integer " + + "column to a long column, a float column to a double column, etc. This is automatically" + + "enabled when reading from Iceberg tables.") + .booleanConf + .createWithDefault(false) +} + +object ConfigHelpers { + def toNumber[T](s: String, converter: String => T, key: String, configType: String): T = { + try { + converter(s.trim) + } catch { + case _: NumberFormatException => + throw new IllegalArgumentException(s"$key should be $configType, but was $s") + } + } + + def toBoolean(s: String, key: String): Boolean = { + try { + s.trim.toBoolean + } catch { + case _: IllegalArgumentException => + throw new IllegalArgumentException(s"$key should be boolean, but was $s") + } + } + + def stringToSeq[T](str: String, converter: String => T): Seq[T] = { + Utils.stringToSeq(str).map(converter) + } + + def seqToString[T](v: Seq[T], stringConverter: T => String): String = { + v.map(stringConverter).mkString(",") + } + + def timeFromString(str: String, unit: TimeUnit): Long = JavaUtils.timeStringAs(str, unit) + + def timeToString(v: Long, unit: TimeUnit): String = + TimeUnit.MILLISECONDS.convert(v, unit) + "ms" + + def byteFromString(str: String, unit: ByteUnit): Long = { + val (input, multiplier) = + if (str.nonEmpty && str.charAt(0) == '-') { + (str.substring(1), -1) + } else { + (str, 1) + } + multiplier * JavaUtils.byteStringAs(input, unit) + } + + def byteToString(v: Long, unit: ByteUnit): String = unit.convertTo(v, ByteUnit.BYTE) + "b" +} + +private class TypedConfigBuilder[T]( + val parent: ConfigBuilder, + val converter: String => T, + val stringConverter: T => String) { + + import ConfigHelpers._ + + def this(parent: ConfigBuilder, converter: String => T) = { + this(parent, converter, Option(_).map(_.toString).orNull) + } + + /** Apply a transformation to the user-provided values of the config entry. */ + def transform(fn: T => T): TypedConfigBuilder[T] = { + new TypedConfigBuilder(parent, s => fn(converter(s)), stringConverter) + } + + /** Checks if the user-provided value for the config matches the validator. */ + def checkValue(validator: T => Boolean, errorMsg: String): TypedConfigBuilder[T] = { + transform { v => + if (!validator(v)) { + throw new IllegalArgumentException(s"'$v' in ${parent.key} is invalid. $errorMsg") + } + v + } + } + + /** Check that user-provided values for the config match a pre-defined set. */ + def checkValues(validValues: Set[T]): TypedConfigBuilder[T] = { + transform { v => + if (!validValues.contains(v)) { + throw new IllegalArgumentException( + s"The value of ${parent.key} should be one of ${validValues.mkString(", ")}, but was $v") + } + v + } + } + + /** Turns the config entry into a sequence of values of the underlying type. */ + def toSequence: TypedConfigBuilder[Seq[T]] = { + new TypedConfigBuilder(parent, stringToSeq(_, converter), seqToString(_, stringConverter)) + } + + /** Creates a [[ConfigEntry]] that does not have a default value. */ + def createOptional: OptionalConfigEntry[T] = { + new OptionalConfigEntry[T]( + parent.key, + converter, + stringConverter, + parent._doc, + parent._public, + parent._version) + } + + /** Creates a [[ConfigEntry]] that has a default value. */ + def createWithDefault(default: T): ConfigEntry[T] = { + val transformedDefault = converter(stringConverter(default)) + new ConfigEntryWithDefault[T]( + parent.key, + transformedDefault, + converter, + stringConverter, + parent._doc, + parent._public, + parent._version) + } +} + +private[comet] abstract class ConfigEntry[T]( + val key: String, + val valueConverter: String => T, + val stringConverter: T => String, + val doc: String, + val isPublic: Boolean, + val version: String) { + + /** + * Retrieves the config value from the given [[SQLConf]]. + */ + def get(conf: SQLConf): T + + /** + * Retrieves the config value from the current thread-local [[SQLConf]] + * @return + */ + def get(): T = get(SQLConf.get) + + def defaultValue: Option[T] = None + def defaultValueString: String + + override def toString: String = { + s"ConfigEntry(key=$key, defaultValue=$defaultValueString, doc=$doc, " + + s"public=$isPublic, version=$version)" + } +} + +private[comet] class ConfigEntryWithDefault[T]( + key: String, + _defaultValue: T, + valueConverter: String => T, + stringConverter: T => String, + doc: String, + isPublic: Boolean, + version: String) + extends ConfigEntry(key, valueConverter, stringConverter, doc, isPublic, version) { + override def defaultValue: Option[T] = Some(_defaultValue) + override def defaultValueString: String = stringConverter(_defaultValue) + + def get(conf: SQLConf): T = { + val tmp = conf.getConfString(key, null) + if (tmp == null) { + _defaultValue + } else { + valueConverter(tmp) + } + } +} + +private[comet] class OptionalConfigEntry[T]( + key: String, + val rawValueConverter: String => T, + val rawStringConverter: T => String, + doc: String, + isPublic: Boolean, + version: String) + extends ConfigEntry[Option[T]]( + key, + s => Some(rawValueConverter(s)), + v => v.map(rawStringConverter).orNull, + doc, + isPublic, + version) { + + override def defaultValueString: String = ConfigEntry.UNDEFINED + + override def get(conf: SQLConf): Option[T] = { + Option(conf.getConfString(key, null)).map(rawValueConverter) + } +} + +private[comet] case class ConfigBuilder(key: String) { + import ConfigHelpers._ + + var _public = true + var _doc = "" + var _version = "" + + def internal(): ConfigBuilder = { + _public = false + this + } + + def doc(s: String): ConfigBuilder = { + _doc = s + this + } + + def version(v: String): ConfigBuilder = { + _version = v + this + } + + def intConf: TypedConfigBuilder[Int] = { + new TypedConfigBuilder(this, toNumber(_, _.toInt, key, "int")) + } + + def longConf: TypedConfigBuilder[Long] = { + new TypedConfigBuilder(this, toNumber(_, _.toLong, key, "long")) + } + + def doubleConf: TypedConfigBuilder[Double] = { + new TypedConfigBuilder(this, toNumber(_, _.toDouble, key, "double")) + } + + def booleanConf: TypedConfigBuilder[Boolean] = { + new TypedConfigBuilder(this, toBoolean(_, key)) + } + + def stringConf: TypedConfigBuilder[String] = { + new TypedConfigBuilder(this, v => v) + } + + def timeConf(unit: TimeUnit): TypedConfigBuilder[Long] = { + new TypedConfigBuilder(this, timeFromString(_, unit), timeToString(_, unit)) + } + + def bytesConf(unit: ByteUnit): TypedConfigBuilder[Long] = { + new TypedConfigBuilder(this, byteFromString(_, unit), byteToString(_, unit)) + } +} + +private object ConfigEntry { + val UNDEFINED = "" +} diff --git a/common/src/main/scala/org/apache/comet/Constants.scala b/common/src/main/scala/org/apache/comet/Constants.scala new file mode 100644 index 000000000..83b570fc3 --- /dev/null +++ b/common/src/main/scala/org/apache/comet/Constants.scala @@ -0,0 +1,25 @@ +/* + * 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.comet + +object Constants { + val LOG_CONF_PATH = "comet.log.file.path" + val LOG_CONF_NAME = "log4rs.yaml" +} diff --git a/common/src/main/scala/org/apache/comet/package.scala b/common/src/main/scala/org/apache/comet/package.scala new file mode 100644 index 000000000..c9aca7538 --- /dev/null +++ b/common/src/main/scala/org/apache/comet/package.scala @@ -0,0 +1,87 @@ +/* + * 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 + +import java.util.Properties + +package object comet { + + /** + * Provides access to build information about the Comet libraries. This will be used by the + * benchmarking software to provide the source revision and repository. In addition, the build + * information is included to aid in future debugging efforts for releases. + */ + private object CometBuildInfo { + + val ( + cometVersion: String, + cometBranch: String, + cometRevision: String, + cometBuildUserName: String, + cometBuildUserEmail: String, + cometRepoUrl: String, + cometBuildTimestamp: String) = { + val resourceStream = Thread + .currentThread() + .getContextClassLoader + .getResourceAsStream("comet-git-info.properties") + if (resourceStream == null) { + throw new CometRuntimeException("Could not find comet-git-info.properties") + } + + try { + val unknownProp = "" + val props = new Properties() + props.load(resourceStream) + ( + props.getProperty("git.build.version", unknownProp), + props.getProperty("git.branch", unknownProp), + props.getProperty("git.commit.id.full", unknownProp), + props.getProperty("git.build.user.name", unknownProp), + props.getProperty("git.build.user.email", unknownProp), + props.getProperty("git.remote.origin.url", unknownProp), + props.getProperty("git.build.time", unknownProp)) + } catch { + case e: Exception => + throw new CometRuntimeException( + "Error loading properties from comet-git-info.properties", + e) + } finally { + if (resourceStream != null) { + try { + resourceStream.close() + } catch { + case e: Exception => + throw new CometRuntimeException("Error closing Comet build info resource stream", e) + } + } + } + } + } + + val COMET_VERSION = CometBuildInfo.cometVersion + val COMET_BRANCH = CometBuildInfo.cometBranch + val COMET_REVISION = CometBuildInfo.cometRevision + val COMET_BUILD_USER_EMAIL = CometBuildInfo.cometBuildUserEmail + val COMET_BUILD_USER_NAME = CometBuildInfo.cometBuildUserName + val COMET_REPO_URL = CometBuildInfo.cometRepoUrl + val COMET_BUILD_TIMESTAMP = CometBuildInfo.cometBuildTimestamp + +} diff --git a/common/src/main/scala/org/apache/comet/parquet/CometParquetUtils.scala b/common/src/main/scala/org/apache/comet/parquet/CometParquetUtils.scala new file mode 100644 index 000000000..d851067b5 --- /dev/null +++ b/common/src/main/scala/org/apache/comet/parquet/CometParquetUtils.scala @@ -0,0 +1,99 @@ +/* + * 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.comet.parquet + +import org.apache.hadoop.conf.Configuration +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +object CometParquetUtils { + private val PARQUET_FIELD_ID_WRITE_ENABLED = "spark.sql.parquet.fieldId.write.enabled" + private val PARQUET_FIELD_ID_READ_ENABLED = "spark.sql.parquet.fieldId.read.enabled" + private val IGNORE_MISSING_PARQUET_FIELD_ID = "spark.sql.parquet.fieldId.read.ignoreMissing" + + def writeFieldId(conf: SQLConf): Boolean = + conf.getConfString(PARQUET_FIELD_ID_WRITE_ENABLED, "false").toBoolean + + def writeFieldId(conf: Configuration): Boolean = + conf.getBoolean(PARQUET_FIELD_ID_WRITE_ENABLED, false) + + def readFieldId(conf: SQLConf): Boolean = + conf.getConfString(PARQUET_FIELD_ID_READ_ENABLED, "false").toBoolean + + def ignoreMissingIds(conf: SQLConf): Boolean = + conf.getConfString(IGNORE_MISSING_PARQUET_FIELD_ID, "false").toBoolean + + // The following is copied from QueryExecutionErrors + // TODO: remove after dropping Spark 3.2.0 support and directly use + // QueryExecutionErrors.foundDuplicateFieldInFieldIdLookupModeError + def foundDuplicateFieldInFieldIdLookupModeError( + requiredId: Int, + matchedFields: String): Throwable = { + new RuntimeException(s""" + |Found duplicate field(s) "$requiredId": $matchedFields + |in id mapping mode + """.stripMargin.replaceAll("\n", " ")) + } + + // The followings are copied from org.apache.spark.sql.execution.datasources.parquet.ParquetUtils + // TODO: remove after dropping Spark 3.2.0 support and directly use ParquetUtils + /** + * A StructField metadata key used to set the field id of a column in the Parquet schema. + */ + val FIELD_ID_METADATA_KEY = "parquet.field.id" + + /** + * Whether there exists a field in the schema, whether inner or leaf, has the parquet field ID + * metadata. + */ + def hasFieldIds(schema: StructType): Boolean = { + def recursiveCheck(schema: DataType): Boolean = { + schema match { + case st: StructType => + st.exists(field => hasFieldId(field) || recursiveCheck(field.dataType)) + + case at: ArrayType => recursiveCheck(at.elementType) + + case mt: MapType => recursiveCheck(mt.keyType) || recursiveCheck(mt.valueType) + + case _ => + // No need to really check primitive types, just to terminate the recursion + false + } + } + if (schema.isEmpty) false else recursiveCheck(schema) + } + + def hasFieldId(field: StructField): Boolean = + field.metadata.contains(FIELD_ID_METADATA_KEY) + + def getFieldId(field: StructField): Int = { + require( + hasFieldId(field), + s"The key `$FIELD_ID_METADATA_KEY` doesn't exist in the metadata of " + field) + try { + Math.toIntExact(field.metadata.getLong(FIELD_ID_METADATA_KEY)) + } catch { + case _: ArithmeticException | _: ClassCastException => + throw new IllegalArgumentException( + s"The key `$FIELD_ID_METADATA_KEY` must be a 32-bit integer") + } + } +} diff --git a/common/src/main/scala/org/apache/comet/parquet/CometReaderThreadPool.scala b/common/src/main/scala/org/apache/comet/parquet/CometReaderThreadPool.scala new file mode 100644 index 000000000..ca13bba0c --- /dev/null +++ b/common/src/main/scala/org/apache/comet/parquet/CometReaderThreadPool.scala @@ -0,0 +1,65 @@ +/* + * 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.comet.parquet + +import java.util.concurrent.{Executors, ExecutorService, ThreadFactory} +import java.util.concurrent.atomic.AtomicLong + +abstract class CometReaderThreadPool { + private var threadPool: Option[ExecutorService] = None + + protected def threadNamePrefix: String + + private def initThreadPool(maxThreads: Int): ExecutorService = synchronized { + if (threadPool.isEmpty) { + val threadFactory: ThreadFactory = new ThreadFactory() { + private val defaultThreadFactory = Executors.defaultThreadFactory + val count = new AtomicLong(0) + + override def newThread(r: Runnable): Thread = { + val thread = defaultThreadFactory.newThread(r) + thread.setName(s"${threadNamePrefix}_${count.getAndIncrement()}") + thread.setDaemon(true) + thread + } + } + + val threadPoolExecutor = Executors.newFixedThreadPool(maxThreads, threadFactory) + threadPool = Some(threadPoolExecutor) + } + + threadPool.get + } + + def getOrCreateThreadPool(numThreads: Int): ExecutorService = { + threadPool.getOrElse(initThreadPool(numThreads)) + } + +} + +// A thread pool used for pre-fetching files. +object CometPrefetchThreadPool extends CometReaderThreadPool { + override def threadNamePrefix: String = "prefetch_thread" +} + +// Thread pool used by the Parquet parallel reader +object CometFileReaderThreadPool extends CometReaderThreadPool { + override def threadNamePrefix: String = "file_reader_thread" +} diff --git a/common/src/main/scala/org/apache/comet/shims/ShimBatchReader.scala b/common/src/main/scala/org/apache/comet/shims/ShimBatchReader.scala new file mode 100644 index 000000000..ece4cfbe5 --- /dev/null +++ b/common/src/main/scala/org/apache/comet/shims/ShimBatchReader.scala @@ -0,0 +1,54 @@ +/* + * 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.comet.shims + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.PartitionedFile + +object ShimBatchReader { + + // TODO: remove after dropping Spark 3.2 & 3.3 support and directly call PartitionedFile + def newPartitionedFile(partitionValues: InternalRow, file: String): PartitionedFile = + classOf[PartitionedFile].getDeclaredConstructors + .map(c => + c.getParameterCount match { + case 5 => + c.newInstance( + partitionValues, + file, + Long.box(-1), // -1 means we read the entire file + Long.box(-1), + Array.empty[String]) + case 7 => + c.newInstance( + partitionValues, + c.getParameterTypes()(1) + .getConstructor(classOf[String]) + .newInstance(file) + .asInstanceOf[AnyRef], + Long.box(-1), // -1 means we read the entire file + Long.box(-1), + Array.empty[String], + Long.box(0), + Long.box(0)) + }) + .head + .asInstanceOf[PartitionedFile] +} diff --git a/common/src/main/scala/org/apache/comet/shims/ShimFileFormat.scala b/common/src/main/scala/org/apache/comet/shims/ShimFileFormat.scala new file mode 100644 index 000000000..5ab7eaf4f --- /dev/null +++ b/common/src/main/scala/org/apache/comet/shims/ShimFileFormat.scala @@ -0,0 +1,35 @@ +/* + * 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.comet.shims + +object ShimFileFormat { + + // TODO: remove after dropping Spark 3.2 & 3.3 support and directly use FileFormat.ROW_INDEX + val ROW_INDEX = "row_index" + + // A name for a temporary column that holds row indexes computed by the file format reader + // until they can be placed in the _metadata struct. + // TODO: remove after dropping Spark 3.2 & 3.3 support and directly use + // FileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME + val ROW_INDEX_TEMPORARY_COLUMN_NAME: String = s"_tmp_metadata_$ROW_INDEX" + + // TODO: remove after dropping Spark 3.2 support and use FileFormat.OPTION_RETURNING_BATCH + val OPTION_RETURNING_BATCH = "returning_batch" +} diff --git a/common/src/main/scala/org/apache/comet/shims/ShimResolveDefaultColumns.scala b/common/src/main/scala/org/apache/comet/shims/ShimResolveDefaultColumns.scala new file mode 100644 index 000000000..8a30c8e00 --- /dev/null +++ b/common/src/main/scala/org/apache/comet/shims/ShimResolveDefaultColumns.scala @@ -0,0 +1,38 @@ +/* + * 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.comet.shims + +import scala.util.Try + +import org.apache.spark.sql.types.{StructField, StructType} + +object ShimResolveDefaultColumns { + // TODO: remove after dropping Spark 3.2 & 3.3 support and directly use ResolveDefaultColumns + def getExistenceDefaultValue(field: StructField): Any = + Try { + // scalastyle:off classforname + Class.forName("org.apache.spark.sql.catalyst.util.ResolveDefaultColumns$") + // scalastyle:on classforname + }.map { objClass => + val objInstance = objClass.getField("MODULE$").get(null) + val method = objClass.getMethod("getExistenceDefaultValues", classOf[StructType]) + method.invoke(objInstance, StructType(Seq(field))).asInstanceOf[Array[Any]].head + }.getOrElse(null) +} diff --git a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala new file mode 100644 index 000000000..1e27ed8f0 --- /dev/null +++ b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala @@ -0,0 +1,141 @@ +/* + * 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.comet.vector + +import scala.collection.mutable + +import org.apache.arrow.c.{ArrowArray, ArrowImporter, ArrowSchema, CDataDictionaryProvider, Data} +import org.apache.arrow.memory.RootAllocator +import org.apache.arrow.vector._ +import org.apache.spark.SparkException +import org.apache.spark.sql.vectorized.ColumnarBatch + +class NativeUtil { + private val allocator = new RootAllocator(Long.MaxValue) + private val dictionaryProvider: CDataDictionaryProvider = new CDataDictionaryProvider + private val importer = new ArrowImporter(allocator) + + /** + * Exports a Comet `ColumnarBatch` into a list of memory addresses that can be consumed by the + * native execution. + * + * @param batch + * the input Comet columnar batch + * @return + * a list containing pairs of memory addresses in the format of (address of Arrow array, + * address of Arrow schema) + */ + def exportBatch(batch: ColumnarBatch): Array[Long] = { + val vectors = (0 until batch.numCols()).flatMap { index => + batch.column(index) match { + case a: CometVector => + val valueVector = a.getValueVector + + val provider = if (valueVector.getField.getDictionary != null) { + a.getDictionaryProvider + } else { + null + } + + val arrowSchema = ArrowSchema.allocateNew(allocator) + val arrowArray = ArrowArray.allocateNew(allocator) + Data.exportVector( + allocator, + getFieldVector(valueVector), + provider, + arrowArray, + arrowSchema) + + Seq((arrowArray, arrowSchema)) + case c => + throw new SparkException( + "Comet execution only takes Arrow Arrays, but got " + + s"${c.getClass}") + } + } + + vectors.flatMap { pair => + Seq(pair._1.memoryAddress(), pair._2.memoryAddress()) + }.toArray + } + + /** + * Imports a list of Arrow addresses from native execution, and return a list of Comet vectors. + * + * @param arrayAddress + * a list containing paris of Arrow addresses from the native, in the format of (address of + * Arrow array, address of Arrow schema) + * @return + * a list of Comet vectors + */ + def importVector(arrayAddress: Array[Long]): Seq[CometVector] = { + val arrayVectors = mutable.ArrayBuffer.empty[CometVector] + + for (i <- arrayAddress.indices by 2) { + val arrowSchema = ArrowSchema.wrap(arrayAddress(i + 1)) + val arrowArray = ArrowArray.wrap(arrayAddress(i)) + + // Native execution should always have 'useDecimal128' set to true since it doesn't support + // other cases. + arrayVectors += CometVector.getVector( + importer.importVector(arrowArray, arrowSchema, dictionaryProvider), + true, + dictionaryProvider) + + arrowArray.close() + arrowSchema.close() + } + arrayVectors.toSeq + } + + /** + * Takes zero-copy slices of the input batch with given start index and maximum number of rows. + * + * @param batch + * Input batch + * @param startIndex + * Start index of the slice + * @param maxNumRows + * Maximum number of rows in the slice + * @return + * A new batch with the sliced vectors + */ + def takeRows(batch: ColumnarBatch, startIndex: Int, maxNumRows: Int): ColumnarBatch = { + val arrayVectors = mutable.ArrayBuffer.empty[CometVector] + + for (i <- 0 until batch.numCols()) { + val column = batch.column(i).asInstanceOf[CometVector] + arrayVectors += column.slice(startIndex, maxNumRows) + } + + new ColumnarBatch(arrayVectors.toArray, maxNumRows) + } + + private def getFieldVector(valueVector: ValueVector): FieldVector = { + valueVector match { + case v @ (_: BitVector | _: TinyIntVector | _: SmallIntVector | _: IntVector | + _: BigIntVector | _: Float4Vector | _: Float8Vector | _: VarCharVector | + _: DecimalVector | _: DateDayVector | _: TimeStampMicroTZVector | _: VarBinaryVector | + _: FixedSizeBinaryVector | _: TimeStampMicroVector) => + v.asInstanceOf[FieldVector] + case _ => throw new SparkException(s"Unsupported Arrow Vector: ${valueVector.getClass}") + } + } +} diff --git a/common/src/main/scala/org/apache/comet/vector/StreamReader.scala b/common/src/main/scala/org/apache/comet/vector/StreamReader.scala new file mode 100644 index 000000000..9c4f99602 --- /dev/null +++ b/common/src/main/scala/org/apache/comet/vector/StreamReader.scala @@ -0,0 +1,70 @@ +/* + * 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.comet.vector + +import java.nio.channels.ReadableByteChannel + +import scala.collection.JavaConverters.collectionAsScalaIterableConverter + +import org.apache.arrow.memory.RootAllocator +import org.apache.arrow.vector.VectorSchemaRoot +import org.apache.arrow.vector.ipc.{ArrowStreamReader, ReadChannel} +import org.apache.arrow.vector.ipc.message.MessageChannelReader +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} + +/** + * A reader that consumes Arrow data from an input channel, and produces Comet batches. + */ +case class StreamReader(channel: ReadableByteChannel) extends AutoCloseable { + private var allocator = new RootAllocator(Long.MaxValue) + private val channelReader = new MessageChannelReader(new ReadChannel(channel), allocator) + private var arrowReader = new ArrowStreamReader(channelReader, allocator) + private var root = arrowReader.getVectorSchemaRoot + + def nextBatch(): Option[ColumnarBatch] = { + if (arrowReader.loadNextBatch()) { + Some(rootAsBatch(root)) + } else { + None + } + } + + private def rootAsBatch(root: VectorSchemaRoot): ColumnarBatch = { + val columns = root.getFieldVectors.asScala.map { vector => + // Native shuffle always uses decimal128. + CometVector.getVector(vector, true, arrowReader).asInstanceOf[ColumnVector] + }.toArray + val batch = new ColumnarBatch(columns) + batch.setNumRows(root.getRowCount) + batch + } + + override def close(): Unit = { + if (root != null) { + arrowReader.close() + root.close() + allocator.close() + + arrowReader = null + root = null + allocator = null + } + } +} diff --git a/common/src/main/scala/org/apache/spark/sql/comet/parquet/CometParquetReadSupport.scala b/common/src/main/scala/org/apache/spark/sql/comet/parquet/CometParquetReadSupport.scala new file mode 100644 index 000000000..0e8a190c2 --- /dev/null +++ b/common/src/main/scala/org/apache/spark/sql/comet/parquet/CometParquetReadSupport.scala @@ -0,0 +1,378 @@ +/* + * 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.spark.sql.comet.parquet + +import java.util.{Locale, UUID} + +import scala.collection.JavaConverters._ + +import org.apache.parquet.schema._ +import org.apache.parquet.schema.LogicalTypeAnnotation.ListLogicalTypeAnnotation +import org.apache.parquet.schema.Type.Repetition +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.types._ + +import org.apache.comet.parquet.CometParquetUtils + +/** + * This class is copied & slightly modified from [[ParquetReadSupport]] in Spark. Changes: + * - This doesn't extend from Parquet's `ReadSupport` class since that is used for row-based + * Parquet reader. Therefore, there is no `init`, `prepareForRead` as well as other methods + * that are unused. + */ +object CometParquetReadSupport { + val SPARK_PARQUET_SCHEMA_NAME = "spark_schema" + + val EMPTY_MESSAGE: MessageType = + Types.buildMessage().named(SPARK_PARQUET_SCHEMA_NAME) + + def generateFakeColumnName: String = s"_fake_name_${UUID.randomUUID()}" + + def clipParquetSchema( + parquetSchema: MessageType, + catalystSchema: StructType, + caseSensitive: Boolean, + useFieldId: Boolean, + ignoreMissingIds: Boolean): MessageType = { + if (!ignoreMissingIds && + !containsFieldIds(parquetSchema) && + CometParquetUtils.hasFieldIds(catalystSchema)) { + throw new RuntimeException( + "Spark read schema expects field Ids, " + + "but Parquet file schema doesn't contain any field Ids.\n" + + "Please remove the field ids from Spark schema or ignore missing ids by " + + "setting `spark.sql.parquet.fieldId.read.ignoreMissing = true`\n" + + s""" + |Spark read schema: + |${catalystSchema.prettyJson} + | + |Parquet file schema: + |${parquetSchema.toString} + |""".stripMargin) + } + clipParquetSchema(parquetSchema, catalystSchema, caseSensitive, useFieldId) + } + + /** + * Tailors `parquetSchema` according to `catalystSchema` by removing column paths don't exist in + * `catalystSchema`, and adding those only exist in `catalystSchema`. + */ + def clipParquetSchema( + parquetSchema: MessageType, + catalystSchema: StructType, + caseSensitive: Boolean, + useFieldId: Boolean): MessageType = { + val clippedParquetFields = clipParquetGroupFields( + parquetSchema.asGroupType(), + catalystSchema, + caseSensitive, + useFieldId) + if (clippedParquetFields.isEmpty) { + EMPTY_MESSAGE + } else { + Types + .buildMessage() + .addFields(clippedParquetFields: _*) + .named(SPARK_PARQUET_SCHEMA_NAME) + } + } + + private def clipParquetType( + parquetType: Type, + catalystType: DataType, + caseSensitive: Boolean, + useFieldId: Boolean): Type = { + val newParquetType = catalystType match { + case t: ArrayType if !isPrimitiveCatalystType(t.elementType) => + // Only clips array types with nested type as element type. + clipParquetListType(parquetType.asGroupType(), t.elementType, caseSensitive, useFieldId) + + case t: MapType + if !isPrimitiveCatalystType(t.keyType) || + !isPrimitiveCatalystType(t.valueType) => + // Only clips map types with nested key type or value type + clipParquetMapType( + parquetType.asGroupType(), + t.keyType, + t.valueType, + caseSensitive, + useFieldId) + + case t: StructType => + clipParquetGroup(parquetType.asGroupType(), t, caseSensitive, useFieldId) + + case _ => + // UDTs and primitive types are not clipped. For UDTs, a clipped version might not be able + // to be mapped to desired user-space types. So UDTs shouldn't participate schema merging. + parquetType + } + + if (useFieldId && parquetType.getId != null) { + newParquetType.withId(parquetType.getId.intValue()) + } else { + newParquetType + } + } + + /** + * Whether a Catalyst [[DataType]] is primitive. Primitive [[DataType]] is not equivalent to + * [[AtomicType]]. For example, [[CalendarIntervalType]] is primitive, but it's not an + * [[AtomicType]]. + */ + private def isPrimitiveCatalystType(dataType: DataType): Boolean = { + dataType match { + case _: ArrayType | _: MapType | _: StructType => false + case _ => true + } + } + + /** + * Clips a Parquet [[GroupType]] which corresponds to a Catalyst [[ArrayType]]. The element type + * of the [[ArrayType]] should also be a nested type, namely an [[ArrayType]], a [[MapType]], or + * a [[StructType]]. + */ + private def clipParquetListType( + parquetList: GroupType, + elementType: DataType, + caseSensitive: Boolean, + useFieldId: Boolean): Type = { + // Precondition of this method, should only be called for lists with nested element types. + assert(!isPrimitiveCatalystType(elementType)) + + // Unannotated repeated group should be interpreted as required list of required element, so + // list element type is just the group itself. Clip it. + if (parquetList.getLogicalTypeAnnotation == null && + parquetList.isRepetition(Repetition.REPEATED)) { + clipParquetType(parquetList, elementType, caseSensitive, useFieldId) + } else { + assert( + parquetList.getLogicalTypeAnnotation.isInstanceOf[ListLogicalTypeAnnotation], + "Invalid Parquet schema. " + + "Logical type annotation of annotated Parquet lists must be ListLogicalTypeAnnotation: " + + parquetList.toString) + + assert( + parquetList.getFieldCount == 1 && parquetList + .getType(0) + .isRepetition(Repetition.REPEATED), + "Invalid Parquet schema. " + + "LIST-annotated group should only have exactly one repeated field: " + + parquetList) + + // Precondition of this method, should only be called for lists with nested element types. + assert(!parquetList.getType(0).isPrimitive) + + val repeatedGroup = parquetList.getType(0).asGroupType() + + // If the repeated field is a group with multiple fields, or the repeated field is a group + // with one field and is named either "array" or uses the LIST-annotated group's name with + // "_tuple" appended then the repeated type is the element type and elements are required. + // Build a new LIST-annotated group with clipped `repeatedGroup` as element type and the + // only field. + if (repeatedGroup.getFieldCount > 1 || + repeatedGroup.getName == "array" || + repeatedGroup.getName == parquetList.getName + "_tuple") { + Types + .buildGroup(parquetList.getRepetition) + .as(LogicalTypeAnnotation.listType()) + .addField(clipParquetType(repeatedGroup, elementType, caseSensitive, useFieldId)) + .named(parquetList.getName) + } else { + // Otherwise, the repeated field's type is the element type with the repeated field's + // repetition. + val newRepeatedGroup = Types + .repeatedGroup() + .addField( + clipParquetType(repeatedGroup.getType(0), elementType, caseSensitive, useFieldId)) + .named(repeatedGroup.getName) + + val newElementType = if (useFieldId && repeatedGroup.getId != null) { + newRepeatedGroup.withId(repeatedGroup.getId.intValue()) + } else { + newRepeatedGroup + } + + Types + .buildGroup(parquetList.getRepetition) + .as(LogicalTypeAnnotation.listType()) + .addField( + Types + .repeatedGroup() + .addField( + clipParquetType(repeatedGroup.getType(0), elementType, caseSensitive, useFieldId)) + .named(repeatedGroup.getName)) + .addField(newElementType) + .named(parquetList.getName) + } + } + } + + /** + * Clips a Parquet [[GroupType]] which corresponds to a Catalyst [[MapType]]. Either key type or + * value type of the [[MapType]] must be a nested type, namely an [[ArrayType]], a [[MapType]], + * or a [[StructType]]. + */ + private def clipParquetMapType( + parquetMap: GroupType, + keyType: DataType, + valueType: DataType, + caseSensitive: Boolean, + useFieldId: Boolean): GroupType = { + // Precondition of this method, only handles maps with nested key types or value types. + assert(!isPrimitiveCatalystType(keyType) || !isPrimitiveCatalystType(valueType)) + + val repeatedGroup = parquetMap.getType(0).asGroupType() + val parquetKeyType = repeatedGroup.getType(0) + val parquetValueType = repeatedGroup.getType(1) + + val clippedRepeatedGroup = { + val newRepeatedGroup = Types + .repeatedGroup() + .as(repeatedGroup.getLogicalTypeAnnotation) + .addField(clipParquetType(parquetKeyType, keyType, caseSensitive, useFieldId)) + .addField(clipParquetType(parquetValueType, valueType, caseSensitive, useFieldId)) + .named(repeatedGroup.getName) + if (useFieldId && repeatedGroup.getId != null) { + newRepeatedGroup.withId(repeatedGroup.getId.intValue()) + } else { + newRepeatedGroup + } + } + + Types + .buildGroup(parquetMap.getRepetition) + .as(parquetMap.getLogicalTypeAnnotation) + .addField(clippedRepeatedGroup) + .named(parquetMap.getName) + } + + /** + * Clips a Parquet [[GroupType]] which corresponds to a Catalyst [[StructType]]. + * + * @return + * A clipped [[GroupType]], which has at least one field. + * @note + * Parquet doesn't allow creating empty [[GroupType]] instances except for empty + * [[MessageType]]. Because it's legal to construct an empty requested schema for column + * pruning. + */ + private def clipParquetGroup( + parquetRecord: GroupType, + structType: StructType, + caseSensitive: Boolean, + useFieldId: Boolean): GroupType = { + val clippedParquetFields = + clipParquetGroupFields(parquetRecord, structType, caseSensitive, useFieldId) + Types + .buildGroup(parquetRecord.getRepetition) + .as(parquetRecord.getLogicalTypeAnnotation) + .addFields(clippedParquetFields: _*) + .named(parquetRecord.getName) + } + + /** + * Clips a Parquet [[GroupType]] which corresponds to a Catalyst [[StructType]]. + * + * @return + * A list of clipped [[GroupType]] fields, which can be empty. + */ + private def clipParquetGroupFields( + parquetRecord: GroupType, + structType: StructType, + caseSensitive: Boolean, + useFieldId: Boolean): Seq[Type] = { + val toParquet = new CometSparkToParquetSchemaConverter( + writeLegacyParquetFormat = false, + useFieldId = useFieldId) + lazy val caseSensitiveParquetFieldMap = + parquetRecord.getFields.asScala.map(f => f.getName -> f).toMap + lazy val caseInsensitiveParquetFieldMap = + parquetRecord.getFields.asScala.groupBy(_.getName.toLowerCase(Locale.ROOT)) + lazy val idToParquetFieldMap = + parquetRecord.getFields.asScala.filter(_.getId != null).groupBy(f => f.getId.intValue()) + + def matchCaseSensitiveField(f: StructField): Type = { + caseSensitiveParquetFieldMap + .get(f.name) + .map(clipParquetType(_, f.dataType, caseSensitive, useFieldId)) + .getOrElse(toParquet.convertField(f)) + } + + def matchCaseInsensitiveField(f: StructField): Type = { + // Do case-insensitive resolution only if in case-insensitive mode + caseInsensitiveParquetFieldMap + .get(f.name.toLowerCase(Locale.ROOT)) + .map { parquetTypes => + if (parquetTypes.size > 1) { + // Need to fail if there is ambiguity, i.e. more than one field is matched + val parquetTypesString = parquetTypes.map(_.getName).mkString("[", ", ", "]") + throw QueryExecutionErrors.foundDuplicateFieldInCaseInsensitiveModeError( + f.name, + parquetTypesString) + } else { + clipParquetType(parquetTypes.head, f.dataType, caseSensitive, useFieldId) + } + } + .getOrElse(toParquet.convertField(f)) + } + + def matchIdField(f: StructField): Type = { + val fieldId = CometParquetUtils.getFieldId(f) + idToParquetFieldMap + .get(fieldId) + .map { parquetTypes => + if (parquetTypes.size > 1) { + // Need to fail if there is ambiguity, i.e. more than one field is matched + val parquetTypesString = parquetTypes.map(_.getName).mkString("[", ", ", "]") + throw CometParquetUtils.foundDuplicateFieldInFieldIdLookupModeError( + fieldId, + parquetTypesString) + } else { + clipParquetType(parquetTypes.head, f.dataType, caseSensitive, useFieldId) + } + } + .getOrElse { + // When there is no ID match, we use a fake name to avoid a name match by accident + // We need this name to be unique as well, otherwise there will be type conflicts + toParquet.convertField(f.copy(name = generateFakeColumnName)) + } + } + + val shouldMatchById = useFieldId && CometParquetUtils.hasFieldIds(structType) + structType.map { f => + if (shouldMatchById && CometParquetUtils.hasFieldId(f)) { + matchIdField(f) + } else if (caseSensitive) { + matchCaseSensitiveField(f) + } else { + matchCaseInsensitiveField(f) + } + } + } + + /** + * Whether the parquet schema contains any field IDs. + */ + private def containsFieldIds(schema: Type): Boolean = schema match { + case p: PrimitiveType => p.getId != null + // We don't require all fields to have IDs, so we use `exists` here. + case g: GroupType => g.getId != null || g.getFields.asScala.exists(containsFieldIds) + } +} diff --git a/common/src/main/scala/org/apache/spark/sql/comet/parquet/CometSparkToParquetSchemaConverter.scala b/common/src/main/scala/org/apache/spark/sql/comet/parquet/CometSparkToParquetSchemaConverter.scala new file mode 100644 index 000000000..2c8187e18 --- /dev/null +++ b/common/src/main/scala/org/apache/spark/sql/comet/parquet/CometSparkToParquetSchemaConverter.scala @@ -0,0 +1,322 @@ +/* + * 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.spark.sql.comet.parquet + +import org.apache.hadoop.conf.Configuration +import org.apache.parquet.schema._ +import org.apache.parquet.schema.LogicalTypeAnnotation._ +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ +import org.apache.parquet.schema.Type.Repetition._ +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +import org.apache.comet.parquet.CometParquetUtils + +/** + * This class is copied & modified from Spark's [[SparkToParquetSchemaConverter]] class. + */ +class CometSparkToParquetSchemaConverter( + writeLegacyParquetFormat: Boolean = SQLConf.PARQUET_WRITE_LEGACY_FORMAT.defaultValue.get, + outputTimestampType: SQLConf.ParquetOutputTimestampType.Value = + SQLConf.ParquetOutputTimestampType.INT96, + useFieldId: Boolean = CometParquetUtils.writeFieldId(new SQLConf)) { + + def this(conf: SQLConf) = this( + writeLegacyParquetFormat = conf.writeLegacyParquetFormat, + outputTimestampType = conf.parquetOutputTimestampType, + useFieldId = CometParquetUtils.writeFieldId(conf)) + + def this(conf: Configuration) = this( + writeLegacyParquetFormat = conf.get(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key).toBoolean, + outputTimestampType = SQLConf.ParquetOutputTimestampType.withName( + conf.get(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key)), + useFieldId = CometParquetUtils.writeFieldId(conf)) + + /** + * Converts a Spark SQL [[StructType]] to a Parquet [[MessageType]]. + */ + def convert(catalystSchema: StructType): MessageType = { + Types + .buildMessage() + .addFields(catalystSchema.map(convertField): _*) + .named(ParquetSchemaConverter.SPARK_PARQUET_SCHEMA_NAME) + } + + /** + * Converts a Spark SQL [[StructField]] to a Parquet [[Type]]. + */ + def convertField(field: StructField): Type = { + val converted = convertField(field, if (field.nullable) OPTIONAL else REQUIRED) + if (useFieldId && CometParquetUtils.hasFieldId(field)) { + converted.withId(CometParquetUtils.getFieldId(field)) + } else { + converted + } + } + + private def convertField(field: StructField, repetition: Type.Repetition): Type = { + + field.dataType match { + // =================== + // Simple atomic types + // =================== + + case BooleanType => + Types.primitive(BOOLEAN, repetition).named(field.name) + + case ByteType => + Types + .primitive(INT32, repetition) + .as(LogicalTypeAnnotation.intType(8, true)) + .named(field.name) + + case ShortType => + Types + .primitive(INT32, repetition) + .as(LogicalTypeAnnotation.intType(16, true)) + .named(field.name) + + case IntegerType => + Types.primitive(INT32, repetition).named(field.name) + + case LongType => + Types.primitive(INT64, repetition).named(field.name) + + case FloatType => + Types.primitive(FLOAT, repetition).named(field.name) + + case DoubleType => + Types.primitive(DOUBLE, repetition).named(field.name) + + case StringType => + Types + .primitive(BINARY, repetition) + .as(LogicalTypeAnnotation.stringType()) + .named(field.name) + + case DateType => + Types + .primitive(INT32, repetition) + .as(LogicalTypeAnnotation.dateType()) + .named(field.name) + + // NOTE: Spark SQL can write timestamp values to Parquet using INT96, TIMESTAMP_MICROS or + // TIMESTAMP_MILLIS. TIMESTAMP_MICROS is recommended but INT96 is the default to keep the + // behavior same as before. + // + // As stated in PARQUET-323, Parquet `INT96` was originally introduced to represent nanosecond + // timestamp in Impala for some historical reasons. It's not recommended to be used for any + // other types and will probably be deprecated in some future version of parquet-format spec. + // That's the reason why parquet-format spec only defines `TIMESTAMP_MILLIS` and + // `TIMESTAMP_MICROS` which are both logical types annotating `INT64`. + // + // Originally, Spark SQL uses the same nanosecond timestamp type as Impala and Hive. Starting + // from Spark 1.5.0, we resort to a timestamp type with microsecond precision so that we can + // store a timestamp into a `Long`. This design decision is subject to change though, for + // example, we may resort to nanosecond precision in the future. + case TimestampType => + outputTimestampType match { + case SQLConf.ParquetOutputTimestampType.INT96 => + Types.primitive(INT96, repetition).named(field.name) + case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS => + Types + .primitive(INT64, repetition) + .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MICROS)) + .named(field.name) + case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MILLIS => + Types + .primitive(INT64, repetition) + .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MILLIS)) + .named(field.name) + } + + case TimestampNTZType => + Types + .primitive(INT64, repetition) + .as(LogicalTypeAnnotation.timestampType(false, TimeUnit.MICROS)) + .named(field.name) + + case BinaryType => + Types.primitive(BINARY, repetition).named(field.name) + + // ====================== + // Decimals (legacy mode) + // ====================== + + // Spark 1.4.x and prior versions only support decimals with a maximum precision of 18 and + // always store decimals in fixed-length byte arrays. To keep compatibility with these older + // versions, here we convert decimals with all precisions to `FIXED_LEN_BYTE_ARRAY` annotated + // by `DECIMAL`. + case DecimalType.Fixed(precision, scale) if writeLegacyParquetFormat => + Types + .primitive(FIXED_LEN_BYTE_ARRAY, repetition) + .as(LogicalTypeAnnotation.decimalType(scale, precision)) + .length(Decimal.minBytesForPrecision(precision)) + .named(field.name) + + // ======================== + // Decimals (standard mode) + // ======================== + + // Uses INT32 for 1 <= precision <= 9 + case DecimalType.Fixed(precision, scale) + if precision <= Decimal.MAX_INT_DIGITS && !writeLegacyParquetFormat => + Types + .primitive(INT32, repetition) + .as(LogicalTypeAnnotation.decimalType(scale, precision)) + .named(field.name) + + // Uses INT64 for 1 <= precision <= 18 + case DecimalType.Fixed(precision, scale) + if precision <= Decimal.MAX_LONG_DIGITS && !writeLegacyParquetFormat => + Types + .primitive(INT64, repetition) + .as(LogicalTypeAnnotation.decimalType(scale, precision)) + .named(field.name) + + // Uses FIXED_LEN_BYTE_ARRAY for all other precisions + case DecimalType.Fixed(precision, scale) if !writeLegacyParquetFormat => + Types + .primitive(FIXED_LEN_BYTE_ARRAY, repetition) + .as(LogicalTypeAnnotation.decimalType(scale, precision)) + .length(Decimal.minBytesForPrecision(precision)) + .named(field.name) + + // =================================== + // ArrayType and MapType (legacy mode) + // =================================== + + // Spark 1.4.x and prior versions convert `ArrayType` with nullable elements into a 3-level + // `LIST` structure. This behavior is somewhat a hybrid of parquet-hive and parquet-avro + // (1.6.0rc3): the 3-level structure is similar to parquet-hive while the 3rd level element + // field name "array" is borrowed from parquet-avro. + case ArrayType(elementType, nullable @ true) if writeLegacyParquetFormat => + // group (LIST) { + // optional group bag { + // repeated array; + // } + // } + + // This should not use `listOfElements` here because this new method checks if the + // element name is `element` in the `GroupType` and throws an exception if not. + // As mentioned above, Spark prior to 1.4.x writes `ArrayType` as `LIST` but with + // `array` as its element name as below. Therefore, we build manually + // the correct group type here via the builder. (See SPARK-16777) + Types + .buildGroup(repetition) + .as(LogicalTypeAnnotation.listType()) + .addField( + Types + .buildGroup(REPEATED) + // "array" is the name chosen by parquet-hive (1.7.0 and prior version) + .addField(convertField(StructField("array", elementType, nullable))) + .named("bag")) + .named(field.name) + + // Spark 1.4.x and prior versions convert ArrayType with non-nullable elements into a 2-level + // LIST structure. This behavior mimics parquet-avro (1.6.0rc3). Note that this case is + // covered by the backwards-compatibility rules implemented in `isElementType()`. + case ArrayType(elementType, nullable @ false) if writeLegacyParquetFormat => + // group (LIST) { + // repeated element; + // } + + // Here too, we should not use `listOfElements`. (See SPARK-16777) + Types + .buildGroup(repetition) + .as(LogicalTypeAnnotation.listType()) + // "array" is the name chosen by parquet-avro (1.7.0 and prior version) + .addField(convertField(StructField("array", elementType, nullable), REPEATED)) + .named(field.name) + + // Spark 1.4.x and prior versions convert MapType into a 3-level group annotated by + // MAP_KEY_VALUE. This is covered by `convertGroupField(field: GroupType): DataType`. + case MapType(keyType, valueType, valueContainsNull) if writeLegacyParquetFormat => + // group (MAP) { + // repeated group map (MAP_KEY_VALUE) { + // required key; + // value; + // } + // } + ConversionPatterns.mapType( + repetition, + field.name, + convertField(StructField("key", keyType, nullable = false)), + convertField(StructField("value", valueType, valueContainsNull))) + + // ===================================== + // ArrayType and MapType (standard mode) + // ===================================== + + case ArrayType(elementType, containsNull) if !writeLegacyParquetFormat => + // group (LIST) { + // repeated group list { + // element; + // } + // } + Types + .buildGroup(repetition) + .as(LogicalTypeAnnotation.listType()) + .addField( + Types + .repeatedGroup() + .addField(convertField(StructField("element", elementType, containsNull))) + .named("list")) + .named(field.name) + + case MapType(keyType, valueType, valueContainsNull) => + // group (MAP) { + // repeated group key_value { + // required key; + // value; + // } + // } + Types + .buildGroup(repetition) + .as(LogicalTypeAnnotation.mapType()) + .addField( + Types + .repeatedGroup() + .addField(convertField(StructField("key", keyType, nullable = false))) + .addField(convertField(StructField("value", valueType, valueContainsNull))) + .named("key_value")) + .named(field.name) + + // =========== + // Other types + // =========== + + case StructType(fields) => + fields + .foldLeft(Types.buildGroup(repetition)) { (builder, field) => + builder.addField(convertField(field)) + } + .named(field.name) + + case udt: UserDefinedType[_] => + convertField(field.copy(dataType = udt.sqlType)) + + case _ => + throw QueryCompilationErrors.cannotConvertDataTypeToParquetTypeError(field) + } + } +} diff --git a/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala b/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala new file mode 100644 index 000000000..9e0541f44 --- /dev/null +++ b/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala @@ -0,0 +1,164 @@ +/* + * 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.spark.sql.comet.util + +import java.io.File + +import scala.collection.JavaConverters._ + +import org.apache.arrow.vector.complex.MapVector +import org.apache.arrow.vector.types._ +import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema} +import org.apache.spark.sql.types._ + +object Utils { + def getConfPath(confFileName: String): String = { + sys.env + .get("COMET_CONF_DIR") + .map { t => new File(s"$t${File.separator}$confFileName") } + .filter(_.isFile) + .map(_.getAbsolutePath) + .orNull + } + + def stringToSeq(str: String): Seq[String] = { + str.split(",").map(_.trim()).filter(_.nonEmpty) + } + + def fromArrowField(field: Field): DataType = { + field.getType match { + case _: ArrowType.Map => + val elementField = field.getChildren.get(0) + val keyType = fromArrowField(elementField.getChildren.get(0)) + val valueType = fromArrowField(elementField.getChildren.get(1)) + MapType(keyType, valueType, elementField.getChildren.get(1).isNullable) + case ArrowType.List.INSTANCE => + val elementField = field.getChildren().get(0) + val elementType = fromArrowField(elementField) + ArrayType(elementType, containsNull = elementField.isNullable) + case ArrowType.Struct.INSTANCE => + val fields = field.getChildren().asScala.map { child => + val dt = fromArrowField(child) + StructField(child.getName, dt, child.isNullable) + } + StructType(fields.toSeq) + case arrowType => fromArrowType(arrowType) + } + } + + def fromArrowType(dt: ArrowType): DataType = dt match { + case ArrowType.Bool.INSTANCE => BooleanType + case int: ArrowType.Int if int.getIsSigned && int.getBitWidth == 8 => ByteType + case int: ArrowType.Int if int.getIsSigned && int.getBitWidth == 8 * 2 => ShortType + case int: ArrowType.Int if int.getIsSigned && int.getBitWidth == 8 * 4 => IntegerType + case int: ArrowType.Int if int.getIsSigned && int.getBitWidth == 8 * 8 => LongType + case float: ArrowType.FloatingPoint if float.getPrecision == FloatingPointPrecision.SINGLE => + FloatType + case float: ArrowType.FloatingPoint if float.getPrecision == FloatingPointPrecision.DOUBLE => + DoubleType + case ArrowType.Utf8.INSTANCE => StringType + case ArrowType.Binary.INSTANCE => BinaryType + case _: ArrowType.FixedSizeBinary => BinaryType + case d: ArrowType.Decimal => DecimalType(d.getPrecision, d.getScale) + case date: ArrowType.Date if date.getUnit == DateUnit.DAY => DateType + case ts: ArrowType.Timestamp if ts.getUnit == TimeUnit.MICROSECOND => TimestampType + case ArrowType.Null.INSTANCE => NullType + case yi: ArrowType.Interval if yi.getUnit == IntervalUnit.YEAR_MONTH => + YearMonthIntervalType() + case di: ArrowType.Interval if di.getUnit == IntervalUnit.DAY_TIME => DayTimeIntervalType() + case _ => throw new UnsupportedOperationException(s"Unsupported data type: ${dt.toString}") + } + + /** Maps data type from Spark to Arrow. NOTE: timeZoneId required for TimestampTypes */ + def toArrowType(dt: DataType, timeZoneId: String): ArrowType = + dt match { + case BooleanType => ArrowType.Bool.INSTANCE + case ByteType => new ArrowType.Int(8, true) + case ShortType => new ArrowType.Int(8 * 2, true) + case IntegerType => new ArrowType.Int(8 * 4, true) + case LongType => new ArrowType.Int(8 * 8, true) + case FloatType => new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE) + case DoubleType => new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE) + case StringType => ArrowType.Utf8.INSTANCE + case BinaryType => ArrowType.Binary.INSTANCE + case DecimalType.Fixed(precision, scale) => new ArrowType.Decimal(precision, scale, 128) + case DateType => new ArrowType.Date(DateUnit.DAY) + case TimestampType => + if (timeZoneId == null) { + throw new UnsupportedOperationException( + s"${TimestampType.catalogString} must supply timeZoneId parameter") + } else { + new ArrowType.Timestamp(TimeUnit.MICROSECOND, timeZoneId) + } + case TimestampNTZType => + new ArrowType.Timestamp(TimeUnit.MICROSECOND, null) + case _ => + throw new UnsupportedOperationException(s"Unsupported data type: ${dt.catalogString}") + } + + /** Maps field from Spark to Arrow. NOTE: timeZoneId required for TimestampType */ + def toArrowField(name: String, dt: DataType, nullable: Boolean, timeZoneId: String): Field = { + dt match { + case ArrayType(elementType, containsNull) => + val fieldType = new FieldType(nullable, ArrowType.List.INSTANCE, null) + new Field( + name, + fieldType, + Seq(toArrowField("element", elementType, containsNull, timeZoneId)).asJava) + case StructType(fields) => + val fieldType = new FieldType(nullable, ArrowType.Struct.INSTANCE, null) + new Field( + name, + fieldType, + fields + .map { field => + toArrowField(field.name, field.dataType, field.nullable, timeZoneId) + } + .toSeq + .asJava) + case MapType(keyType, valueType, valueContainsNull) => + val mapType = new FieldType(nullable, new ArrowType.Map(false), null) + // Note: Map Type struct can not be null, Struct Type key field can not be null + new Field( + name, + mapType, + Seq( + toArrowField( + MapVector.DATA_VECTOR_NAME, + new StructType() + .add(MapVector.KEY_NAME, keyType, nullable = false) + .add(MapVector.VALUE_NAME, valueType, nullable = valueContainsNull), + nullable = false, + timeZoneId)).asJava) + case dataType => + val fieldType = new FieldType(nullable, toArrowType(dataType, timeZoneId), null) + new Field(name, fieldType, Seq.empty[Field].asJava) + } + } + + /** + * Maps schema from Spark to Arrow. NOTE: timeZoneId required for TimestampType in StructType + */ + def toArrowSchema(schema: StructType, timeZoneId: String): Schema = { + new Schema(schema.map { field => + toArrowField(field.name, field.dataType, field.nullable, timeZoneId) + }.asJava) + } +} diff --git a/common/src/test/java/org/apache/comet/parquet/TestColumnReader.java b/common/src/test/java/org/apache/comet/parquet/TestColumnReader.java new file mode 100644 index 000000000..d4e748a9b --- /dev/null +++ b/common/src/test/java/org/apache/comet/parquet/TestColumnReader.java @@ -0,0 +1,193 @@ +/* + * 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.comet.parquet; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.function.BiFunction; + +import scala.collection.JavaConverters; + +import org.junit.Test; + +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.FixedSizeBinaryVector; +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.VarBinaryVector; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.types.*; +import org.apache.spark.sql.vectorized.ColumnVector; + +import org.apache.comet.vector.CometPlainVector; +import org.apache.comet.vector.CometVector; + +import static org.apache.spark.sql.types.DataTypes.*; +import static org.junit.Assert.*; + +@SuppressWarnings("unchecked") +public class TestColumnReader { + private static final int BATCH_SIZE = 1024; + private static final List TYPES = + Arrays.asList( + BooleanType, + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + BinaryType, + DecimalType.apply(5, 2), + DecimalType.apply(18, 10), + DecimalType.apply(19, 5)); + private static final List VALUES = + Arrays.asList( + true, + (byte) 42, + (short) 100, + 1000, + (long) 10000, + (float) 3.14, + 3.1415926, + new byte[] {1, 2, 3, 4, 5, 6, 7, 8}, + Decimal.apply("123.45"), + Decimal.apply("00.0123456789"), + Decimal.apply("-001234.56789")); + private static final List> GETTERS = + Arrays.asList( + ColumnVector::getBoolean, + ColumnVector::getByte, + ColumnVector::getShort, + ColumnVector::getInt, + ColumnVector::getLong, + ColumnVector::getFloat, + ColumnVector::getDouble, + ColumnVector::getBinary, + (v, i) -> v.getDecimal(i, 5, 2), + (v, i) -> v.getDecimal(i, 18, 10), + (v, i) -> v.getDecimal(i, 19, 5)); + + @Test + public void testConstantVectors() { + for (int i = 0; i < TYPES.size(); i++) { + DataType type = TYPES.get(i); + StructField field = StructField.apply("f", type, false, null); + + List values = Collections.singletonList(VALUES.get(i)); + InternalRow row = GenericInternalRow.apply(JavaConverters.asScalaBuffer(values).toSeq()); + ConstantColumnReader reader = new ConstantColumnReader(field, BATCH_SIZE, row, 0, true); + reader.readBatch(BATCH_SIZE); + CometVector vector = reader.currentBatch(); + assertEquals(BATCH_SIZE, vector.numValues()); + assertEquals(0, vector.numNulls()); + for (int j = 0; j < BATCH_SIZE; j++) { + if (TYPES.get(i) == BinaryType || TYPES.get(i) == StringType) { + assertArrayEquals((byte[]) VALUES.get(i), (byte[]) GETTERS.get(i).apply(vector, j)); + } else { + assertEquals(VALUES.get(i), GETTERS.get(i).apply(vector, j)); + } + } + + // Test null values too + row.setNullAt(0); + reader = new ConstantColumnReader(field, BATCH_SIZE, row, 0, true); + reader.readBatch(BATCH_SIZE); + vector = reader.currentBatch(); + assertEquals(BATCH_SIZE, vector.numValues()); + assertEquals(BATCH_SIZE, vector.numNulls()); + for (int j = 0; j < BATCH_SIZE; j++) { + assertTrue(vector.isNullAt(j)); + } + } + + if (org.apache.spark.package$.MODULE$.SPARK_VERSION_SHORT().compareTo("3.4") >= 0) { + Metadata meta = new MetadataBuilder().putString("EXISTS_DEFAULT", "123").build(); + StructField field = StructField.apply("f", LongType, false, meta); + ConstantColumnReader reader = new ConstantColumnReader(field, BATCH_SIZE, true); + reader.readBatch(BATCH_SIZE); + CometVector vector = reader.currentBatch(); + + assertEquals(BATCH_SIZE, vector.numValues()); + assertEquals(0, vector.numNulls()); + for (int j = 0; j < BATCH_SIZE; j++) { + assertEquals(123, vector.getLong(j)); + } + } + } + + @Test + public void testRowIndexColumnVectors() { + StructField field = StructField.apply("f", LongType, false, null); + int bigBatchSize = BATCH_SIZE * 2; + int step = 4; + int batchSize = bigBatchSize / step; + long[] indices = new long[step * 2]; + List expected = new ArrayList<>(); + + long idx = 0, len = 0; + for (int i = 0; i < step; i++) { + idx = ThreadLocalRandom.current().nextLong(idx + len, Long.MAX_VALUE); + indices[i * 2] = idx; + len = ThreadLocalRandom.current().nextLong(Long.max(bigBatchSize - expected.size(), 0)); + indices[i * 2 + 1] = len; + for (int j = 0; j < len; j++) { + expected.add(idx + j); + } + } + + RowIndexColumnReader reader = new RowIndexColumnReader(field, BATCH_SIZE, indices); + for (int i = 0; i < step; i++) { + reader.readBatch(batchSize); + CometVector vector = reader.currentBatch(); + assertEquals( + Integer.min(batchSize, Integer.max(expected.size() - i * batchSize, 0)), + vector.numValues()); + assertEquals(0, vector.numNulls()); + for (int j = 0; j < vector.numValues(); j++) { + assertEquals((long) expected.get(i * batchSize + j), vector.getLong(j)); + } + } + + reader.close(); + } + + @Test + public void testIsFixedLength() { + BufferAllocator allocator = new RootAllocator(Integer.MAX_VALUE); + + ValueVector vv = new IntVector("v1", allocator); + CometVector vector = new CometPlainVector(vv, false); + assertTrue(vector.isFixedLength()); + + vv = new FixedSizeBinaryVector("v2", allocator, 12); + vector = new CometPlainVector(vv, false); + assertTrue(vector.isFixedLength()); + + vv = new VarBinaryVector("v3", allocator); + vector = new CometPlainVector(vv, false); + assertFalse(vector.isFixedLength()); + } +} diff --git a/common/src/test/java/org/apache/comet/parquet/TestCometInputFile.java b/common/src/test/java/org/apache/comet/parquet/TestCometInputFile.java new file mode 100644 index 000000000..63bb65d5c --- /dev/null +++ b/common/src/test/java/org/apache/comet/parquet/TestCometInputFile.java @@ -0,0 +1,44 @@ +/* + * 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.comet.parquet; + +import org.junit.Assert; +import org.junit.Test; + +public class TestCometInputFile { + @Test + public void testIsAtLeastHadoop33() { + Assert.assertTrue(CometInputFile.isAtLeastHadoop33("3.3.0")); + Assert.assertTrue(CometInputFile.isAtLeastHadoop33("3.4.0-SNAPSHOT")); + Assert.assertTrue(CometInputFile.isAtLeastHadoop33("3.12.5")); + Assert.assertTrue(CometInputFile.isAtLeastHadoop33("3.20.6.4-xyz")); + + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("2.7.2")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("2.7.3-SNAPSHOT")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("2.7")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("2")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3.2")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3.0.2.5-abc")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3.1.2-test")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3-SNAPSHOT")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3.2-SNAPSHOT")); + } +} diff --git a/common/src/test/java/org/apache/comet/parquet/TestFileReader.java b/common/src/test/java/org/apache/comet/parquet/TestFileReader.java new file mode 100644 index 000000000..6e73f7510 --- /dev/null +++ b/common/src/test/java/org/apache/comet/parquet/TestFileReader.java @@ -0,0 +1,829 @@ +/* + * 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.comet.parquet; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Method; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.*; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.HadoopReadOptions; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.bytes.BytesUtils; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.page.DataPage; +import org.apache.parquet.column.page.DataPageV1; +import org.apache.parquet.column.page.DataPageV2; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.statistics.BinaryStatistics; +import org.apache.parquet.column.statistics.Statistics; +import org.apache.parquet.column.values.bloomfilter.BlockSplitBloomFilter; +import org.apache.parquet.column.values.bloomfilter.BloomFilter; +import org.apache.parquet.filter2.predicate.FilterApi; +import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.filter2.predicate.Operators; +import org.apache.parquet.hadoop.ParquetFileWriter; +import org.apache.parquet.hadoop.ParquetInputFormat; +import org.apache.parquet.hadoop.metadata.*; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.internal.column.columnindex.BoundaryOrder; +import org.apache.parquet.internal.column.columnindex.ColumnIndex; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.io.InputFile; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.MessageTypeParser; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Types; + +import static org.apache.parquet.column.Encoding.*; +import static org.apache.parquet.format.converter.ParquetMetadataConverter.MAX_STATS_SIZE; +import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; + +@SuppressWarnings("deprecation") +public class TestFileReader { + private static final MessageType SCHEMA = + MessageTypeParser.parseMessageType( + "" + + "message m {" + + " required group a {" + + " required binary b;" + + " }" + + " required group c {" + + " required int64 d;" + + " }" + + "}"); + + private static final MessageType SCHEMA2 = + MessageTypeParser.parseMessageType( + "" + + "message root { " + + "required int32 id;" + + "required binary name(UTF8); " + + "required int32 num; " + + "required binary comment(UTF8);" + + "}"); + + private static final MessageType PROJECTED_SCHEMA2 = + MessageTypeParser.parseMessageType( + "" + + "message root { " + + "required int32 id;" + + "required binary name(UTF8); " + + "required binary comment(UTF8);" + + "}"); + + private static final String[] PATH1 = {"a", "b"}; + private static final ColumnDescriptor C1 = SCHEMA.getColumnDescription(PATH1); + private static final String[] PATH2 = {"c", "d"}; + private static final ColumnDescriptor C2 = SCHEMA.getColumnDescription(PATH2); + + private static final byte[] BYTES1 = {0, 1, 2, 3}; + private static final byte[] BYTES2 = {1, 2, 3, 4}; + private static final byte[] BYTES3 = {2, 3, 4, 5}; + private static final byte[] BYTES4 = {3, 4, 5, 6}; + private static final CompressionCodecName CODEC = CompressionCodecName.UNCOMPRESSED; + + private static final org.apache.parquet.column.statistics.Statistics EMPTY_STATS = + org.apache.parquet.column.statistics.Statistics.getBuilderForReading( + Types.required(PrimitiveTypeName.BINARY).named("test_binary")) + .build(); + + @Rule public final TemporaryFolder temp = new TemporaryFolder(); + + @Test + public void testEnableReadParallel() { + Configuration configuration = new Configuration(); + ReadOptions options = ReadOptions.builder(configuration).build(); + + assertFalse(FileReader.shouldReadParallel(options, "hdfs")); + assertFalse(FileReader.shouldReadParallel(options, "file")); + assertFalse(FileReader.shouldReadParallel(options, null)); + assertTrue(FileReader.shouldReadParallel(options, "s3a")); + + options = ReadOptions.builder(configuration).enableParallelIO(false).build(); + assertFalse(FileReader.shouldReadParallel(options, "s3a")); + } + + @Test + public void testReadWrite() throws Exception { + File testFile = temp.newFile(); + testFile.delete(); + + Path path = new Path(testFile.toURI()); + Configuration configuration = new Configuration(); + + // Start a Parquet file with 2 row groups, each with 2 column chunks + ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path); + w.start(); + w.startBlock(3); + w.startColumn(C1, 5, CODEC); + long c1Starts = w.getPos(); + long c1p1Starts = w.getPos(); + w.writeDataPage(2, 4, BytesInput.from(BYTES1), EMPTY_STATS, 2, RLE, RLE, PLAIN); + w.writeDataPage(3, 4, BytesInput.from(BYTES1), EMPTY_STATS, 3, RLE, RLE, PLAIN); + w.endColumn(); + long c1Ends = w.getPos(); + w.startColumn(C2, 6, CODEC); + long c2Starts = w.getPos(); + w.writeDictionaryPage(new DictionaryPage(BytesInput.from(BYTES2), 4, RLE_DICTIONARY)); + long c2p1Starts = w.getPos(); + w.writeDataPage(2, 4, BytesInput.from(BYTES2), EMPTY_STATS, 2, RLE, RLE, PLAIN); + w.writeDataPage(3, 4, BytesInput.from(BYTES2), EMPTY_STATS, 3, RLE, RLE, PLAIN); + w.writeDataPage(1, 4, BytesInput.from(BYTES2), EMPTY_STATS, 1, RLE, RLE, PLAIN); + w.endColumn(); + long c2Ends = w.getPos(); + w.endBlock(); + w.startBlock(4); + w.startColumn(C1, 7, CODEC); + w.writeDataPage(7, 4, BytesInput.from(BYTES3), EMPTY_STATS, 7, RLE, RLE, PLAIN); + w.endColumn(); + w.startColumn(C2, 8, CODEC); + w.writeDataPage(8, 4, BytesInput.from(BYTES4), EMPTY_STATS, 8, RLE, RLE, PLAIN); + w.endColumn(); + w.endBlock(); + w.end(new HashMap<>()); + + InputFile file = HadoopInputFile.fromPath(path, configuration); + ParquetReadOptions options = ParquetReadOptions.builder().build(); + ReadOptions cometOptions = ReadOptions.builder(configuration).build(); + + try (FileReader reader = new FileReader(file, options, cometOptions)) { + ParquetMetadata readFooter = reader.getFooter(); + assertEquals("footer: " + readFooter, 2, readFooter.getBlocks().size()); + BlockMetaData rowGroup = readFooter.getBlocks().get(0); + assertEquals(c1Ends - c1Starts, rowGroup.getColumns().get(0).getTotalSize()); + assertEquals(c2Ends - c2Starts, rowGroup.getColumns().get(1).getTotalSize()); + assertEquals(c2Ends - c1Starts, rowGroup.getTotalByteSize()); + + assertEquals(c1Starts, rowGroup.getColumns().get(0).getStartingPos()); + assertEquals(0, rowGroup.getColumns().get(0).getDictionaryPageOffset()); + assertEquals(c1p1Starts, rowGroup.getColumns().get(0).getFirstDataPageOffset()); + assertEquals(c2Starts, rowGroup.getColumns().get(1).getStartingPos()); + assertEquals(c2Starts, rowGroup.getColumns().get(1).getDictionaryPageOffset()); + assertEquals(c2p1Starts, rowGroup.getColumns().get(1).getFirstDataPageOffset()); + + HashSet expectedEncoding = new HashSet<>(); + expectedEncoding.add(PLAIN); + expectedEncoding.add(RLE); + assertEquals(expectedEncoding, rowGroup.getColumns().get(0).getEncodings()); + } + + // read first block of col #1 + try (FileReader r = new FileReader(file, options, cometOptions)) { + r.setRequestedSchema(Arrays.asList(SCHEMA.getColumnDescription(PATH1))); + PageReadStore pages = r.readNextRowGroup(); + assertEquals(3, pages.getRowCount()); + validateContains(pages, PATH1, 2, BytesInput.from(BYTES1)); + validateContains(pages, PATH1, 3, BytesInput.from(BYTES1)); + assertTrue(r.skipNextRowGroup()); + assertNull(r.readNextRowGroup()); + } + + // read all blocks of col #1 and #2 + try (FileReader r = new FileReader(file, options, cometOptions)) { + r.setRequestedSchema( + Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2))); + PageReadStore pages = r.readNextRowGroup(); + assertEquals(3, pages.getRowCount()); + validateContains(pages, PATH1, 2, BytesInput.from(BYTES1)); + validateContains(pages, PATH1, 3, BytesInput.from(BYTES1)); + validateContains(pages, PATH2, 2, BytesInput.from(BYTES2)); + validateContains(pages, PATH2, 3, BytesInput.from(BYTES2)); + validateContains(pages, PATH2, 1, BytesInput.from(BYTES2)); + + pages = r.readNextRowGroup(); + assertEquals(4, pages.getRowCount()); + + validateContains(pages, PATH1, 7, BytesInput.from(BYTES3)); + validateContains(pages, PATH2, 8, BytesInput.from(BYTES4)); + + assertNull(r.readNextRowGroup()); + } + } + + @Test + public void testBloomFilterReadWrite() throws Exception { + MessageType schema = + MessageTypeParser.parseMessageType("message test { required binary foo; }"); + File testFile = temp.newFile(); + testFile.delete(); + Path path = new Path(testFile.toURI()); + Configuration configuration = new Configuration(); + configuration.set("parquet.bloom.filter.column.names", "foo"); + String[] colPath = {"foo"}; + + ColumnDescriptor col = schema.getColumnDescription(colPath); + BinaryStatistics stats1 = new BinaryStatistics(); + ParquetFileWriter w = new ParquetFileWriter(configuration, schema, path); + w.start(); + w.startBlock(3); + w.startColumn(col, 5, CODEC); + w.writeDataPage(2, 4, BytesInput.from(BYTES1), stats1, 2, RLE, RLE, PLAIN); + w.writeDataPage(3, 4, BytesInput.from(BYTES1), stats1, 2, RLE, RLE, PLAIN); + w.endColumn(); + BloomFilter blockSplitBloomFilter = new BlockSplitBloomFilter(0); + blockSplitBloomFilter.insertHash(blockSplitBloomFilter.hash(Binary.fromString("hello"))); + blockSplitBloomFilter.insertHash(blockSplitBloomFilter.hash(Binary.fromString("world"))); + addBloomFilter(w, "foo", blockSplitBloomFilter); + w.endBlock(); + w.end(new HashMap<>()); + + InputFile file = HadoopInputFile.fromPath(path, configuration); + ParquetReadOptions options = ParquetReadOptions.builder().build(); + ReadOptions cometOptions = ReadOptions.builder(configuration).build(); + + try (FileReader r = new FileReader(file, options, cometOptions)) { + ParquetMetadata footer = r.getFooter(); + r.setRequestedSchema(Arrays.asList(schema.getColumnDescription(colPath))); + BloomFilterReader bloomFilterReader = + new BloomFilterReader( + footer.getBlocks().get(0), + r.getFileMetaData().getFileDecryptor(), + r.getInputStream()); + BloomFilter bloomFilter = + bloomFilterReader.readBloomFilter(footer.getBlocks().get(0).getColumns().get(0)); + assertTrue(bloomFilter.findHash(blockSplitBloomFilter.hash(Binary.fromString("hello")))); + assertTrue(bloomFilter.findHash(blockSplitBloomFilter.hash(Binary.fromString("world")))); + } + } + + @Test + public void testReadWriteDataPageV2() throws Exception { + File testFile = temp.newFile(); + testFile.delete(); + + Path path = new Path(testFile.toURI()); + Configuration configuration = new Configuration(); + + ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path); + w.start(); + w.startBlock(14); + + BytesInput repLevels = BytesInput.fromInt(2); + BytesInput defLevels = BytesInput.fromInt(1); + BytesInput data = BytesInput.fromInt(3); + BytesInput data2 = BytesInput.fromInt(10); + + org.apache.parquet.column.statistics.Statistics statsC1P1 = createStatistics("s", "z", C1); + org.apache.parquet.column.statistics.Statistics statsC1P2 = createStatistics("b", "d", C1); + + w.startColumn(C1, 6, CODEC); + long c1Starts = w.getPos(); + w.writeDataPageV2(4, 1, 3, repLevels, defLevels, PLAIN, data, 4, statsC1P1); + w.writeDataPageV2(3, 0, 3, repLevels, defLevels, PLAIN, data, 4, statsC1P2); + w.endColumn(); + long c1Ends = w.getPos(); + + w.startColumn(C2, 5, CODEC); + long c2Starts = w.getPos(); + w.writeDataPageV2(5, 2, 3, repLevels, defLevels, PLAIN, data2, 4, EMPTY_STATS); + w.writeDataPageV2(2, 0, 2, repLevels, defLevels, PLAIN, data2, 4, EMPTY_STATS); + w.endColumn(); + long c2Ends = w.getPos(); + + w.endBlock(); + w.end(new HashMap<>()); + + InputFile file = HadoopInputFile.fromPath(path, configuration); + ParquetReadOptions options = ParquetReadOptions.builder().build(); + ReadOptions cometOptions = ReadOptions.builder(configuration).build(); + + try (FileReader reader = new FileReader(file, options, cometOptions)) { + ParquetMetadata footer = reader.getFooter(); + assertEquals("footer: " + footer, 1, footer.getBlocks().size()); + assertEquals(c1Ends - c1Starts, footer.getBlocks().get(0).getColumns().get(0).getTotalSize()); + assertEquals(c2Ends - c2Starts, footer.getBlocks().get(0).getColumns().get(1).getTotalSize()); + assertEquals(c2Ends - c1Starts, footer.getBlocks().get(0).getTotalByteSize()); + + // check for stats + org.apache.parquet.column.statistics.Statistics expectedStats = + createStatistics("b", "z", C1); + assertStatsValuesEqual( + expectedStats, footer.getBlocks().get(0).getColumns().get(0).getStatistics()); + + HashSet expectedEncoding = new HashSet<>(); + expectedEncoding.add(PLAIN); + assertEquals(expectedEncoding, footer.getBlocks().get(0).getColumns().get(0).getEncodings()); + } + + try (FileReader r = new FileReader(file, options, cometOptions)) { + r.setRequestedSchema( + Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2))); + PageReadStore pages = r.readNextRowGroup(); + assertEquals(14, pages.getRowCount()); + validateV2Page( + pages, + PATH1, + 3, + 4, + 1, + repLevels.toByteArray(), + defLevels.toByteArray(), + data.toByteArray(), + 12); + validateV2Page( + pages, + PATH1, + 3, + 3, + 0, + repLevels.toByteArray(), + defLevels.toByteArray(), + data.toByteArray(), + 12); + validateV2Page( + pages, + PATH2, + 3, + 5, + 2, + repLevels.toByteArray(), + defLevels.toByteArray(), + data2.toByteArray(), + 12); + validateV2Page( + pages, + PATH2, + 2, + 2, + 0, + repLevels.toByteArray(), + defLevels.toByteArray(), + data2.toByteArray(), + 12); + assertNull(r.readNextRowGroup()); + } + } + + @Test + public void testColumnIndexFilter() throws Exception { + File testFile = temp.newFile(); + testFile.delete(); + + Path path = new Path(testFile.toURI()); + Configuration configuration = new Configuration(); + + ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path); + + w.start(); + w.startBlock(4); + w.startColumn(C1, 7, CODEC); + w.writeDataPage(2, 4, BytesInput.from(BYTES1), EMPTY_STATS, 2, RLE, RLE, PLAIN); + w.writeDataPage(2, 4, BytesInput.from(BYTES2), EMPTY_STATS, 2, RLE, RLE, PLAIN); + w.endColumn(); + w.startColumn(C2, 8, CODEC); + // the first page contains one matching record + w.writeDataPage(1, 4, BytesInput.from(BYTES3), statsC2(2L), 1, RLE, RLE, PLAIN); + // all the records of the second page are larger than 2, so should be filtered out + w.writeDataPage(3, 4, BytesInput.from(BYTES4), statsC2(3L, 4L, 5L), 3, RLE, RLE, PLAIN); + w.endColumn(); + w.endBlock(); + + w.startBlock(4); + w.startColumn(C1, 7, CODEC); + w.writeDataPage(2, 4, BytesInput.from(BYTES1), EMPTY_STATS, 2, RLE, RLE, PLAIN); + w.writeDataPage(2, 4, BytesInput.from(BYTES2), EMPTY_STATS, 2, RLE, RLE, PLAIN); + w.endColumn(); + w.startColumn(C2, 8, CODEC); + // the first page should be filtered out + w.writeDataPage(1, 4, BytesInput.from(BYTES3), statsC2(4L), 1, RLE, RLE, PLAIN); + // the second page will be read since it contains matching record + w.writeDataPage(3, 4, BytesInput.from(BYTES4), statsC2(0L, 1L, 3L), 3, RLE, RLE, PLAIN); + w.endColumn(); + w.endBlock(); + + w.end(new HashMap<>()); + + // set a simple equality filter in the ParquetInputFormat + Operators.LongColumn c2 = FilterApi.longColumn("c.d"); + FilterPredicate p = FilterApi.eq(c2, 2L); + ParquetInputFormat.setFilterPredicate(configuration, p); + InputFile file = HadoopInputFile.fromPath(path, configuration); + ParquetReadOptions options = HadoopReadOptions.builder(configuration).build(); + ReadOptions cometOptions = ReadOptions.builder(configuration).build(); + + try (FileReader r = new FileReader(file, options, cometOptions)) { + assertEquals(4, r.getFilteredRecordCount()); + PageReadStore readStore = r.readNextFilteredRowGroup(); + + PageReader c1Reader = readStore.getPageReader(C1); + List c1Pages = new ArrayList<>(); + DataPage page; + while ((page = c1Reader.readPage()) != null) { + c1Pages.add(page); + } + // second page of c1 should be filtered out + assertEquals(1, c1Pages.size()); + validatePage(c1Pages.get(0), 2, BytesInput.from(BYTES1)); + + PageReader c2Reader = readStore.getPageReader(C2); + List c2Pages = new ArrayList<>(); + while ((page = c2Reader.readPage()) != null) { + c2Pages.add(page); + } + assertEquals(1, c2Pages.size()); + validatePage(c2Pages.get(0), 1, BytesInput.from(BYTES3)); + + // test the second row group + readStore = r.readNextFilteredRowGroup(); + assertNotNull(readStore); + + c1Reader = readStore.getPageReader(C1); + c1Pages.clear(); + while ((page = c1Reader.readPage()) != null) { + c1Pages.add(page); + } + // all pages of c1 should be retained + assertEquals(2, c1Pages.size()); + validatePage(c1Pages.get(0), 2, BytesInput.from(BYTES1)); + validatePage(c1Pages.get(1), 2, BytesInput.from(BYTES2)); + + c2Reader = readStore.getPageReader(C2); + c2Pages.clear(); + while ((page = c2Reader.readPage()) != null) { + c2Pages.add(page); + } + assertEquals(1, c2Pages.size()); + validatePage(c2Pages.get(0), 3, BytesInput.from(BYTES4)); + } + } + + @Test + public void testColumnIndexReadWrite() throws Exception { + File testFile = temp.newFile(); + testFile.delete(); + + Path path = new Path(testFile.toURI()); + Configuration configuration = new Configuration(); + + ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path); + w.start(); + w.startBlock(4); + w.startColumn(C1, 7, CODEC); + w.writeDataPage(7, 4, BytesInput.from(BYTES3), EMPTY_STATS, RLE, RLE, PLAIN); + w.endColumn(); + w.startColumn(C2, 8, CODEC); + w.writeDataPage(8, 4, BytesInput.from(BYTES4), EMPTY_STATS, RLE, RLE, PLAIN); + w.endColumn(); + w.endBlock(); + w.startBlock(4); + w.startColumn(C1, 5, CODEC); + long c1p1Starts = w.getPos(); + w.writeDataPage( + 2, 4, BytesInput.from(BYTES1), statsC1(null, Binary.fromString("aaa")), 1, RLE, RLE, PLAIN); + long c1p2Starts = w.getPos(); + w.writeDataPage( + 3, + 4, + BytesInput.from(BYTES1), + statsC1(Binary.fromString("bbb"), Binary.fromString("ccc")), + 3, + RLE, + RLE, + PLAIN); + w.endColumn(); + long c1Ends = w.getPos(); + w.startColumn(C2, 6, CODEC); + long c2p1Starts = w.getPos(); + w.writeDataPage(2, 4, BytesInput.from(BYTES2), statsC2(117L, 100L), 1, RLE, RLE, PLAIN); + long c2p2Starts = w.getPos(); + w.writeDataPage(3, 4, BytesInput.from(BYTES2), statsC2(null, null, null), 2, RLE, RLE, PLAIN); + long c2p3Starts = w.getPos(); + w.writeDataPage(1, 4, BytesInput.from(BYTES2), statsC2(0L), 1, RLE, RLE, PLAIN); + w.endColumn(); + long c2Ends = w.getPos(); + w.endBlock(); + w.startBlock(4); + w.startColumn(C1, 7, CODEC); + w.writeDataPage( + 7, + 4, + BytesInput.from(BYTES3), + // Creating huge stats so the column index will reach the limit and won't be written + statsC1( + Binary.fromConstantByteArray(new byte[(int) MAX_STATS_SIZE]), + Binary.fromConstantByteArray(new byte[1])), + 4, + RLE, + RLE, + PLAIN); + w.endColumn(); + w.startColumn(C2, 8, CODEC); + w.writeDataPage(8, 4, BytesInput.from(BYTES4), EMPTY_STATS, RLE, RLE, PLAIN); + w.endColumn(); + w.endBlock(); + w.end(new HashMap<>()); + + InputFile file = HadoopInputFile.fromPath(path, configuration); + ParquetReadOptions options = ParquetReadOptions.builder().build(); + ReadOptions cometOptions = ReadOptions.builder(configuration).build(); + + try (FileReader reader = new FileReader(file, options, cometOptions)) { + ParquetMetadata footer = reader.getFooter(); + assertEquals(3, footer.getBlocks().size()); + BlockMetaData blockMeta = footer.getBlocks().get(1); + assertEquals(2, blockMeta.getColumns().size()); + + ColumnIndexReader indexReader = reader.getColumnIndexReader(1); + ColumnIndex columnIndex = indexReader.readColumnIndex(blockMeta.getColumns().get(0)); + assertEquals(BoundaryOrder.ASCENDING, columnIndex.getBoundaryOrder()); + assertEquals(Arrays.asList(1L, 0L), columnIndex.getNullCounts()); + assertEquals(Arrays.asList(false, false), columnIndex.getNullPages()); + List minValues = columnIndex.getMinValues(); + assertEquals(2, minValues.size()); + List maxValues = columnIndex.getMaxValues(); + assertEquals(2, maxValues.size()); + assertEquals("aaa", new String(minValues.get(0).array(), StandardCharsets.UTF_8)); + assertEquals("aaa", new String(maxValues.get(0).array(), StandardCharsets.UTF_8)); + assertEquals("bbb", new String(minValues.get(1).array(), StandardCharsets.UTF_8)); + assertEquals("ccc", new String(maxValues.get(1).array(), StandardCharsets.UTF_8)); + + columnIndex = indexReader.readColumnIndex(blockMeta.getColumns().get(1)); + assertEquals(BoundaryOrder.DESCENDING, columnIndex.getBoundaryOrder()); + assertEquals(Arrays.asList(0L, 3L, 0L), columnIndex.getNullCounts()); + assertEquals(Arrays.asList(false, true, false), columnIndex.getNullPages()); + minValues = columnIndex.getMinValues(); + assertEquals(3, minValues.size()); + maxValues = columnIndex.getMaxValues(); + assertEquals(3, maxValues.size()); + assertEquals(100, BytesUtils.bytesToLong(minValues.get(0).array())); + assertEquals(117, BytesUtils.bytesToLong(maxValues.get(0).array())); + assertEquals(0, minValues.get(1).array().length); + assertEquals(0, maxValues.get(1).array().length); + assertEquals(0, BytesUtils.bytesToLong(minValues.get(2).array())); + assertEquals(0, BytesUtils.bytesToLong(maxValues.get(2).array())); + + OffsetIndex offsetIndex = indexReader.readOffsetIndex(blockMeta.getColumns().get(0)); + assertEquals(2, offsetIndex.getPageCount()); + assertEquals(c1p1Starts, offsetIndex.getOffset(0)); + assertEquals(c1p2Starts, offsetIndex.getOffset(1)); + assertEquals(c1p2Starts - c1p1Starts, offsetIndex.getCompressedPageSize(0)); + assertEquals(c1Ends - c1p2Starts, offsetIndex.getCompressedPageSize(1)); + assertEquals(0, offsetIndex.getFirstRowIndex(0)); + assertEquals(1, offsetIndex.getFirstRowIndex(1)); + + offsetIndex = indexReader.readOffsetIndex(blockMeta.getColumns().get(1)); + assertEquals(3, offsetIndex.getPageCount()); + assertEquals(c2p1Starts, offsetIndex.getOffset(0)); + assertEquals(c2p2Starts, offsetIndex.getOffset(1)); + assertEquals(c2p3Starts, offsetIndex.getOffset(2)); + assertEquals(c2p2Starts - c2p1Starts, offsetIndex.getCompressedPageSize(0)); + assertEquals(c2p3Starts - c2p2Starts, offsetIndex.getCompressedPageSize(1)); + assertEquals(c2Ends - c2p3Starts, offsetIndex.getCompressedPageSize(2)); + assertEquals(0, offsetIndex.getFirstRowIndex(0)); + assertEquals(1, offsetIndex.getFirstRowIndex(1)); + assertEquals(3, offsetIndex.getFirstRowIndex(2)); + + assertNull(indexReader.readColumnIndex(footer.getBlocks().get(2).getColumns().get(0))); + } + } + + // Test reader with merging of scan ranges enabled + @Test + public void testWriteReadMergeScanRange() throws Throwable { + Configuration conf = new Configuration(); + conf.set(ReadOptions.COMET_IO_MERGE_RANGES, Boolean.toString(true)); + // Set the merge range delta so small that ranges do not get merged + conf.set(ReadOptions.COMET_IO_MERGE_RANGES_DELTA, Integer.toString(1024)); + testReadWrite(conf, 2, 1024); + // Set the merge range delta so large that all ranges get merged + conf.set(ReadOptions.COMET_IO_MERGE_RANGES_DELTA, Integer.toString(1024 * 1024)); + testReadWrite(conf, 2, 1024); + } + + // `addBloomFilter` is package-private in Parquet, so this uses reflection to access it + private void addBloomFilter(ParquetFileWriter w, String s, BloomFilter filter) throws Exception { + Method method = + ParquetFileWriter.class.getDeclaredMethod( + "addBloomFilter", String.class, BloomFilter.class); + method.setAccessible(true); + method.invoke(w, s, filter); + } + + private void validateContains(PageReadStore pages, String[] path, int values, BytesInput bytes) + throws IOException { + PageReader pageReader = pages.getPageReader(SCHEMA.getColumnDescription(path)); + DataPage page = pageReader.readPage(); + validatePage(page, values, bytes); + } + + private void validatePage(DataPage page, int values, BytesInput bytes) throws IOException { + assertEquals(values, page.getValueCount()); + assertArrayEquals(bytes.toByteArray(), ((DataPageV1) page).getBytes().toByteArray()); + } + + private void validateV2Page( + PageReadStore pages, + String[] path, + int values, + int rows, + int nullCount, + byte[] repetition, + byte[] definition, + byte[] data, + int uncompressedSize) + throws IOException { + PageReader pageReader = pages.getPageReader(SCHEMA.getColumnDescription(path)); + DataPageV2 page = (DataPageV2) pageReader.readPage(); + assertEquals(values, page.getValueCount()); + assertEquals(rows, page.getRowCount()); + assertEquals(nullCount, page.getNullCount()); + assertEquals(uncompressedSize, page.getUncompressedSize()); + assertArrayEquals(repetition, page.getRepetitionLevels().toByteArray()); + assertArrayEquals(definition, page.getDefinitionLevels().toByteArray()); + assertArrayEquals(data, page.getData().toByteArray()); + } + + private Statistics createStatistics(String min, String max, ColumnDescriptor col) { + return Statistics.getBuilderForReading(col.getPrimitiveType()) + .withMin(Binary.fromString(min).getBytes()) + .withMax(Binary.fromString(max).getBytes()) + .withNumNulls(0) + .build(); + } + + public static void assertStatsValuesEqual(Statistics expected, Statistics actual) { + if (expected == actual) { + return; + } + if (expected == null || actual == null) { + assertEquals(expected, actual); + } + Assert.assertArrayEquals(expected.getMaxBytes(), actual.getMaxBytes()); + Assert.assertArrayEquals(expected.getMinBytes(), actual.getMinBytes()); + Assert.assertEquals(expected.getNumNulls(), actual.getNumNulls()); + } + + private Statistics statsC1(Binary... values) { + Statistics stats = Statistics.createStats(C1.getPrimitiveType()); + for (Binary value : values) { + if (value == null) { + stats.incrementNumNulls(); + } else { + stats.updateStats(value); + } + } + return stats; + } + + /** + * Generates arbitrary data for simple schemas, writes the data to a file and also returns the + * data. + * + * @return array of data pages for each column + */ + private HashMap generateAndWriteData( + Configuration configuration, + Path path, + MessageType schema, + int numPages, + int numRecordsPerPage) + throws IOException { + + HashMap dataPages = new HashMap<>(); + + Generator generator = new Generator(); + ParquetFileWriter writer = new ParquetFileWriter(configuration, schema, path); + writer.start(); + writer.startBlock((long) numPages * numRecordsPerPage); + for (ColumnDescriptor colDesc : schema.getColumns()) { + writer.startColumn(colDesc, (long) numPages * numRecordsPerPage, CODEC); + String type = colDesc.getPrimitiveType().getName(); + byte[][] allPages = new byte[numPages][]; + byte[] data; + for (int i = 0; i < numPages; i++) { + data = generator.generateValues(numRecordsPerPage, type); + writer.writeDataPage( + numRecordsPerPage, + data.length, + BytesInput.from(data), + EMPTY_STATS, + numRecordsPerPage, + RLE, + RLE, + PLAIN); + allPages[i] = data; + } + dataPages.put(String.join(".", colDesc.getPath()), allPages); + writer.endColumn(); + } + writer.endBlock(); + writer.end(new HashMap<>()); + return dataPages; + } + + private void readAndValidatePageData( + InputFile inputFile, + ParquetReadOptions options, + ReadOptions cometOptions, + MessageType schema, + HashMap expected, + int expectedValuesPerPage) + throws IOException { + try (FileReader fileReader = new FileReader(inputFile, options, cometOptions)) { + fileReader.setRequestedSchema(schema.getColumns()); + PageReadStore pages = fileReader.readNextRowGroup(); + for (ColumnDescriptor colDesc : schema.getColumns()) { + byte[][] allExpectedPages = expected.get(String.join(".", colDesc.getPath())); + PageReader pageReader = pages.getPageReader(colDesc); + for (byte[] expectedPage : allExpectedPages) { + DataPage page = pageReader.readPage(); + validatePage(page, expectedValuesPerPage, BytesInput.from(expectedPage)); + } + } + } + } + + public void testReadWrite(Configuration configuration, int numPages, int numRecordsPerPage) + throws Exception { + File testFile = temp.newFile(); + testFile.delete(); + + Path path = new Path(testFile.toURI()); + HashMap dataPages = + generateAndWriteData(configuration, path, SCHEMA2, numPages, numRecordsPerPage); + InputFile file = HadoopInputFile.fromPath(path, configuration); + ParquetReadOptions options = ParquetReadOptions.builder().build(); + ReadOptions cometOptions = ReadOptions.builder(configuration).build(); + + readAndValidatePageData( + file, options, cometOptions, PROJECTED_SCHEMA2, dataPages, numRecordsPerPage); + } + + static class Generator { + + static Random random = new Random(1729); + private static final String ALPHABET = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz -"; + private static final int STR_MIN_SIZE = 5; + private static final int STR_MAX_SIZE = 30; + + private byte[] getString(int minSize, int maxSize) { + int size = random.nextInt(maxSize - minSize) + minSize; + byte[] str = new byte[size]; + for (int i = 0; i < size; ++i) { + str[i] = (byte) ALPHABET.charAt(random.nextInt(ALPHABET.length())); + } + return str; + } + + private byte[] generateValues(int numValues, String type) throws IOException { + + if (type.equals("int32")) { + byte[] data = new byte[4 * numValues]; + random.nextBytes(data); + return data; + } else { + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + for (int i = 0; i < numValues; i++) { + outputStream.write(getString(STR_MIN_SIZE, STR_MAX_SIZE)); + } + return outputStream.toByteArray(); + } + } + } + + private Statistics statsC2(Long... values) { + Statistics stats = Statistics.createStats(C2.getPrimitiveType()); + for (Long value : values) { + if (value == null) { + stats.incrementNumNulls(); + } else { + stats.updateStats(value); + } + } + return stats; + } +} diff --git a/common/src/test/resources/log4j.properties b/common/src/test/resources/log4j.properties new file mode 100644 index 000000000..2f46ce155 --- /dev/null +++ b/common/src/test/resources/log4j.properties @@ -0,0 +1,36 @@ +# +# 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. +# + +# Set everything to be logged to the file target/unit-tests.log +test.appender=file +log4j.rootCategory=INFO, ${test.appender} +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Tests that launch java subprocesses can set the "test.appender" system property to +# "console" to avoid having the child process's logs overwrite the unit test's +# log file. +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%t: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.sparkproject.jetty=WARN diff --git a/common/src/test/resources/log4j2.properties b/common/src/test/resources/log4j2.properties new file mode 100644 index 000000000..04cdf8533 --- /dev/null +++ b/common/src/test/resources/log4j2.properties @@ -0,0 +1,40 @@ +# 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. + +# Set everything to be logged to the file target/unit-tests.log +rootLogger.level = info +rootLogger.appenderRef.file.ref = ${sys:test.appender:-File} + +appender.file.type = File +appender.file.name = File +appender.file.fileName = target/unit-tests.log +appender.file.layout.type = PatternLayout +appender.file.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Tests that launch java subprocesses can set the "test.appender" system property to +# "console" to avoid having the child process's logs overwrite the unit test's +# log file. +appender.console.type = Console +appender.console.name = console +appender.console.target = SYSTEM_ERR +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %t: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +logger.jetty.name = org.sparkproject.jetty +logger.jetty.level = warn + diff --git a/conf/log4rs.yaml b/conf/log4rs.yaml new file mode 100644 index 000000000..43277918f --- /dev/null +++ b/conf/log4rs.yaml @@ -0,0 +1,26 @@ +# 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. + +appenders: + unittest: + kind: file + path: "target/unit-tests.log" + +root: + level: info + appenders: + - unittest diff --git a/core/Cargo.lock b/core/Cargo.lock new file mode 100644 index 000000000..0585d7ec7 --- /dev/null +++ b/core/Cargo.lock @@ -0,0 +1,3191 @@ +# This file is automatically @generated by Cargo. +# It is not intended for manual editing. +version = 3 + +[[package]] +name = "addr2line" +version = "0.21.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a30b2e23b9e17a9f90641c7ab1549cd9b44f296d3ccbf309d2863cfe398a0cb" +dependencies = [ + "gimli", +] + +[[package]] +name = "adler" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" + +[[package]] +name = "ahash" +version = "0.8.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77c3a9648d43b9cd48db467b3f87fdd6e146bcc88ab0180006cef2179fe11d01" +dependencies = [ + "cfg-if", + "const-random", + "getrandom", + "once_cell", + "version_check", + "zerocopy", +] + +[[package]] +name = "aho-corasick" +version = "1.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b2969dcb958b36655471fc61f7e416fa76033bdd4bfed0678d8fee1e2d07a1f0" +dependencies = [ + "memchr", +] + +[[package]] +name = "alloc-no-stdlib" +version = "2.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc7bb162ec39d46ab1ca8c77bf72e890535becd1751bb45f64c597edb4c8c6b3" + +[[package]] +name = "alloc-stdlib" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94fb8275041c72129eb51b7d0322c29b8387a0386127718b096429201a5d6ece" +dependencies = [ + "alloc-no-stdlib", +] + +[[package]] +name = "allocator-api2" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0942ffc6dcaadf03badf6e6a2d0228460359d5e34b57ccdc720b7382dfbd5ec5" + +[[package]] +name = "android-tzdata" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e999941b234f3131b00bc13c22d06e8c5ff726d1b6318ac7eb276997bbb4fef0" + +[[package]] +name = "android_system_properties" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "819e7219dbd41043ac279b19830f2efc897156490d7fd6ea916720117ee66311" +dependencies = [ + "libc", +] + +[[package]] +name = "anes" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" + +[[package]] +name = "anstyle" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7079075b41f533b8c61d2a4d073c4676e1f8b249ff94a393b0595db304e0dd87" + +[[package]] +name = "anyhow" +version = "1.0.79" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "080e9890a082662b09c1ad45f567faeeb47f22b5fb23895fbe1e651e718e25ca" + +[[package]] +name = "arc-swap" +version = "1.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bddcadddf5e9015d310179a59bb28c4d4b9920ad0f11e8e14dbadf654890c9a6" + +[[package]] +name = "arrayref" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6b4930d2cb77ce62f89ee5d5289b4ac049559b1c45539271f5ed4fdc7db34545" + +[[package]] +name = "arrayvec" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" + +[[package]] +name = "arrow" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5bc25126d18a012146a888a0298f2c22e1150327bd2765fc76d710a556b2d614" +dependencies = [ + "ahash", + "arrow-arith", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-csv", + "arrow-data", + "arrow-ipc", + "arrow-json", + "arrow-ord", + "arrow-row", + "arrow-schema", + "arrow-select", + "arrow-string", +] + +[[package]] +name = "arrow-arith" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34ccd45e217ffa6e53bbb0080990e77113bdd4e91ddb84e97b77649810bcf1a7" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "chrono", + "half 2.1.0", + "num", +] + +[[package]] +name = "arrow-array" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6bda9acea48b25123c08340f3a8ac361aa0f74469bb36f5ee9acf923fce23e9d" +dependencies = [ + "ahash", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "chrono", + "chrono-tz", + "half 2.1.0", + "hashbrown 0.14.3", + "num", +] + +[[package]] +name = "arrow-buffer" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "01a0fc21915b00fc6c2667b069c1b64bdd920982f426079bc4a7cab86822886c" +dependencies = [ + "bytes", + "half 2.1.0", + "num", +] + +[[package]] +name = "arrow-cast" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5dc0368ed618d509636c1e3cc20db1281148190a78f43519487b2daf07b63b4a" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "base64", + "chrono", + "comfy-table", + "half 2.1.0", + "lexical-core", + "num", +] + +[[package]] +name = "arrow-csv" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2e09aa6246a1d6459b3f14baeaa49606cfdbca34435c46320e14054d244987ca" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", + "chrono", + "csv", + "csv-core", + "lazy_static", + "lexical-core", + "regex", +] + +[[package]] +name = "arrow-data" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "907fafe280a3874474678c1858b9ca4cb7fd83fb8034ff5b6d6376205a08c634" +dependencies = [ + "arrow-buffer", + "arrow-schema", + "half 2.1.0", + "num", +] + +[[package]] +name = "arrow-ipc" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "79a43d6808411886b8c7d4f6f7dd477029c1e77ffffffb7923555cc6579639cd" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", + "flatbuffers", +] + +[[package]] +name = "arrow-json" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d82565c91fd627922ebfe2810ee4e8346841b6f9361b87505a9acea38b614fee" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", + "chrono", + "half 2.1.0", + "indexmap 2.1.0", + "lexical-core", + "num", + "serde", + "serde_json", +] + +[[package]] +name = "arrow-ord" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b23b0e53c0db57c6749997fd343d4c0354c994be7eca67152dd2bdb9a3e1bb4" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "half 2.1.0", + "num", +] + +[[package]] +name = "arrow-row" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "361249898d2d6d4a6eeb7484be6ac74977e48da12a4dd81a708d620cc558117a" +dependencies = [ + "ahash", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "half 2.1.0", + "hashbrown 0.14.3", +] + +[[package]] +name = "arrow-schema" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09e28a5e781bf1b0f981333684ad13f5901f4cd2f20589eab7cf1797da8fc167" +dependencies = [ + "bitflags 2.4.1", +] + +[[package]] +name = "arrow-select" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4f6208466590960efc1d2a7172bc4ff18a67d6e25c529381d7f96ddaf0dc4036" +dependencies = [ + "ahash", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "num", +] + +[[package]] +name = "arrow-string" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a4a48149c63c11c9ff571e50ab8f017d2a7cb71037a882b42f6354ed2da9acc7" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "num", + "regex", + "regex-syntax", +] + +[[package]] +name = "assertables" +version = "7.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c24e9d990669fbd16806bff449e4ac644fd9b1fca014760087732fe4102f131" + +[[package]] +name = "async-trait" +version = "0.1.77" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c980ee35e870bd1a4d2c8294d4c04d0499e67bca1e4b5cefcc693c2fa00caea9" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "autocfg" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" + +[[package]] +name = "backtrace" +version = "0.3.69" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2089b7e3f35b9dd2d0ed921ead4f6d318c27680d4a5bd167b3ee120edb105837" +dependencies = [ + "addr2line", + "cc", + "cfg-if", + "libc", + "miniz_oxide", + "object", + "rustc-demangle", +] + +[[package]] +name = "base64" +version = "0.21.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35636a1494ede3b646cc98f74f8e62c773a38a659ebc777a2cf26b9b74171df9" + +[[package]] +name = "bitflags" +version = "1.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" + +[[package]] +name = "bitflags" +version = "2.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "327762f6e5a765692301e5bb513e0d9fef63be86bbc14528052b1cd3e6f03e07" + +[[package]] +name = "blake2" +version = "0.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46502ad458c9a52b69d4d4d32775c788b7a1b85e8bc9d482d92250fc0e3f8efe" +dependencies = [ + "digest", +] + +[[package]] +name = "blake3" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0231f06152bf547e9c2b5194f247cd97aacf6dcd8b15d8e5ec0663f64580da87" +dependencies = [ + "arrayref", + "arrayvec", + "cc", + "cfg-if", + "constant_time_eq", +] + +[[package]] +name = "block-buffer" +version = "0.10.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3078c7629b62d3f0439517fa394996acacc5cbc91c5a20d8c658e77abd503a71" +dependencies = [ + "generic-array", +] + +[[package]] +name = "brotli" +version = "3.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "516074a47ef4bce09577a3b379392300159ce5b1ba2e501ff1c819950066100f" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", + "brotli-decompressor", +] + +[[package]] +name = "brotli-decompressor" +version = "2.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4e2e4afe60d7dd600fdd3de8d0f08c2b7ec039712e3b6137ff98b7004e82de4f" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", +] + +[[package]] +name = "bumpalo" +version = "3.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f30e7476521f6f8af1a1c4c0b8cc94f0bee37d91763d0ca2665f299b6cd8aec" + +[[package]] +name = "bytemuck" +version = "1.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "374d28ec25809ee0e23827c2ab573d729e293f281dfe393500e7ad618baa61c6" + +[[package]] +name = "byteorder" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" + +[[package]] +name = "bytes" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a2bd12c1caf447e69cd4528f47f94d203fd2582878ecb9e9465484c4148a8223" + +[[package]] +name = "cast" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" + +[[package]] +name = "cc" +version = "1.0.83" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1174fb0b6ec23863f8b971027804a42614e347eafb0a95bf0b12cdae21fc4d0" +dependencies = [ + "jobserver", + "libc", +] + +[[package]] +name = "cesu8" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d43a04d8753f35258c91f8ec639f792891f748a1edbd759cf1dcea3382ad83c" + +[[package]] +name = "cfg-if" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" + +[[package]] +name = "chrono" +version = "0.4.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f2c685bad3eb3d45a01354cedb7d5faa66194d1d58ba6e267a8de788f79db38" +dependencies = [ + "android-tzdata", + "iana-time-zone", + "js-sys", + "num-traits", + "wasm-bindgen", + "windows-targets 0.48.5", +] + +[[package]] +name = "chrono-tz" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91d7b79e99bfaa0d47da0687c43aa3b7381938a62ad3a6498599039321f660b7" +dependencies = [ + "chrono", + "chrono-tz-build", + "phf", +] + +[[package]] +name = "chrono-tz-build" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "433e39f13c9a060046954e0592a8d0a4bcb1040125cbf91cb8ee58964cfb350f" +dependencies = [ + "parse-zoneinfo", + "phf", + "phf_codegen", +] + +[[package]] +name = "ciborium" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "effd91f6c78e5a4ace8a5d3c0b6bfaec9e2baaef55f3efc00e45fb2e477ee926" +dependencies = [ + "ciborium-io", + "ciborium-ll", + "serde", +] + +[[package]] +name = "ciborium-io" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cdf919175532b369853f5d5e20b26b43112613fd6fe7aee757e35f7a44642656" + +[[package]] +name = "ciborium-ll" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "defaa24ecc093c77630e6c15e17c51f5e187bf35ee514f4e2d67baaa96dae22b" +dependencies = [ + "ciborium-io", + "half 1.8.2", +] + +[[package]] +name = "clap" +version = "4.4.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "52bdc885e4cacc7f7c9eedc1ef6da641603180c783c41a15c264944deeaab642" +dependencies = [ + "clap_builder", +] + +[[package]] +name = "clap_builder" +version = "4.4.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fb7fb5e4e979aec3be7791562fcba452f94ad85e954da024396433e0e25a79e9" +dependencies = [ + "anstyle", + "clap_lex", +] + +[[package]] +name = "clap_lex" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "702fc72eb24e5a1e48ce58027a675bc24edd52096d5397d4aea7c6dd9eca0bd1" + +[[package]] +name = "combine" +version = "4.6.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35ed6e9d84f0b51a7f52daf1c7d71dd136fd7a3f41a8462b8cdb8c78d920fad4" +dependencies = [ + "bytes", + "memchr", +] + +[[package]] +name = "comet" +version = "0.1.0" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-data", + "arrow-schema", + "arrow-string", + "assertables", + "async-trait", + "brotli", + "byteorder", + "bytes", + "chrono", + "chrono-tz", + "crc32fast", + "criterion", + "datafusion", + "datafusion-common", + "datafusion-physical-expr", + "flate2", + "futures", + "half 2.1.0", + "hashbrown 0.14.3", + "itertools 0.11.0", + "jni", + "lazy_static", + "log", + "log4rs", + "lz4", + "mimalloc", + "num", + "once_cell", + "parking_lot", + "parquet", + "parquet-format", + "paste", + "pprof", + "prost 0.12.3", + "prost-build", + "rand", + "regex", + "serde", + "simd-adler32", + "snap", + "tempfile", + "thiserror", + "thrift 0.17.0", + "tokio", + "tokio-stream", + "unicode-segmentation", + "zstd", +] + +[[package]] +name = "comfy-table" +version = "7.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7c64043d6c7b7a4c58e39e7efccfdea7b93d885a795d0c054a69dbbf4dd52686" +dependencies = [ + "strum", + "strum_macros", + "unicode-width", +] + +[[package]] +name = "const-random" +version = "0.1.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5aaf16c9c2c612020bcfd042e170f6e32de9b9d75adb5277cdbbd2e2c8c8299a" +dependencies = [ + "const-random-macro", +] + +[[package]] +name = "const-random-macro" +version = "0.1.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" +dependencies = [ + "getrandom", + "once_cell", + "tiny-keccak", +] + +[[package]] +name = "constant_time_eq" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f7144d30dcf0fafbce74250a3963025d8d52177934239851c917d29f1df280c2" + +[[package]] +name = "core-foundation-sys" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06ea2b9bc92be3c2baa9334a323ebca2d6f074ff852cd1d7b11064035cd3868f" + +[[package]] +name = "cpp_demangle" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e8227005286ec39567949b33df9896bcadfa6051bccca2488129f108ca23119" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "cpufeatures" +version = "0.2.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "53fe5e26ff1b7aef8bca9c6080520cfb8d9333c7568e1829cef191a9723e5504" +dependencies = [ + "libc", +] + +[[package]] +name = "crc32fast" +version = "1.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b540bd8bc810d3885c6ea91e2018302f68baba2129ab3e88f32389ee9370880d" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "criterion" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2b12d017a929603d80db1831cd3a24082f8137ce19c69e6447f54f5fc8d692f" +dependencies = [ + "anes", + "cast", + "ciborium", + "clap", + "criterion-plot", + "is-terminal", + "itertools 0.10.5", + "num-traits", + "once_cell", + "oorandom", + "plotters", + "rayon", + "regex", + "serde", + "serde_derive", + "serde_json", + "tinytemplate", + "walkdir", +] + +[[package]] +name = "criterion-plot" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6b50826342786a51a89e2da3a28f1c32b06e387201bc2d19791f622c673706b1" +dependencies = [ + "cast", + "itertools 0.10.5", +] + +[[package]] +name = "crossbeam-deque" +version = "0.8.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fca89a0e215bab21874660c67903c5f143333cab1da83d041c7ded6053774751" +dependencies = [ + "cfg-if", + "crossbeam-epoch", + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-epoch" +version = "0.9.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0e3681d554572a651dda4186cd47240627c3d0114d45a95f6ad27f2f22e7548d" +dependencies = [ + "autocfg", + "cfg-if", + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-utils" +version = "0.8.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c3a430a770ebd84726f584a90ee7f020d28db52c6d02138900f22341f866d39c" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "crunchy" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" + +[[package]] +name = "crypto-common" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" +dependencies = [ + "generic-array", + "typenum", +] + +[[package]] +name = "csv" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac574ff4d437a7b5ad237ef331c17ccca63c46479e5b5453eb8e10bb99a759fe" +dependencies = [ + "csv-core", + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "csv-core" +version = "0.1.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5efa2b3d7902f4b634a20cae3c9c4e6209dc4779feb6863329607560143efa70" +dependencies = [ + "memchr", +] + +[[package]] +name = "dashmap" +version = "5.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "978747c1d849a7d2ee5e8adc0159961c48fb7e5db2f06af6723b80123bb53856" +dependencies = [ + "cfg-if", + "hashbrown 0.14.3", + "lock_api", + "once_cell", + "parking_lot_core", +] + +[[package]] +name = "datafusion" +version = "34.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "193fd1e7628278d0641c5122860f9a7fd6a1d77d055838d12f55d15bbe28d4d0" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-schema", + "async-trait", + "bytes", + "chrono", + "dashmap", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-optimizer", + "datafusion-physical-expr", + "datafusion-physical-plan", + "datafusion-sql", + "futures", + "glob", + "half 2.1.0", + "hashbrown 0.14.3", + "indexmap 2.1.0", + "itertools 0.12.0", + "log", + "num_cpus", + "object_store", + "parking_lot", + "pin-project-lite", + "rand", + "sqlparser", + "tempfile", + "tokio", + "tokio-util", + "url", + "uuid", +] + +[[package]] +name = "datafusion-common" +version = "34.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "548bc49c4a489e3de474813831ea556dc9d368f9ed8d867b1493da42e8e9f613" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-schema", + "chrono", + "half 2.1.0", + "libc", + "num_cpus", + "object_store", + "sqlparser", +] + +[[package]] +name = "datafusion-execution" +version = "34.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ecc865657ffcf4da5ff08bdc6436a9a833bc0aa96c3254c8d18ab8a0ad4e437d" +dependencies = [ + "arrow", + "chrono", + "dashmap", + "datafusion-common", + "datafusion-expr", + "futures", + "hashbrown 0.14.3", + "log", + "object_store", + "parking_lot", + "rand", + "tempfile", + "url", +] + +[[package]] +name = "datafusion-expr" +version = "34.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "33c473f72d8d81a532e63f6e562ed66dd9209dfd8e433d9712abd42444ee161e" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "datafusion-common", + "paste", + "sqlparser", + "strum", + "strum_macros", +] + +[[package]] +name = "datafusion-optimizer" +version = "34.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb6218318001d2f6783b7fffa17592318f65f26609d7aab605a3dd0c7c2e2618" +dependencies = [ + "arrow", + "async-trait", + "chrono", + "datafusion-common", + "datafusion-expr", + "datafusion-physical-expr", + "hashbrown 0.14.3", + "itertools 0.12.0", + "log", + "regex-syntax", +] + +[[package]] +name = "datafusion-physical-expr" +version = "34.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9e1ca7e35ca22f9dc506c2375b92054b03ccf91afe25c0a90b395a1473a09735" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-ord", + "arrow-schema", + "base64", + "blake2", + "blake3", + "chrono", + "datafusion-common", + "datafusion-expr", + "half 2.1.0", + "hashbrown 0.14.3", + "hex", + "indexmap 2.1.0", + "itertools 0.12.0", + "log", + "md-5", + "paste", + "petgraph", + "rand", + "regex", + "sha2", + "unicode-segmentation", + "uuid", +] + +[[package]] +name = "datafusion-physical-plan" +version = "34.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ddde97adefcca3a55257c646ffee2a95b6cac66f74d1146a6e3a6dbb37830631" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-schema", + "async-trait", + "chrono", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "futures", + "half 2.1.0", + "hashbrown 0.14.3", + "indexmap 2.1.0", + "itertools 0.12.0", + "log", + "once_cell", + "parking_lot", + "pin-project-lite", + "rand", + "tokio", + "uuid", +] + +[[package]] +name = "datafusion-sql" +version = "34.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a60d9d6460a64fddb8663db41da97e6b8b0bf79da42f997ebe81722731eaf0e5" +dependencies = [ + "arrow", + "arrow-schema", + "datafusion-common", + "datafusion-expr", + "log", + "sqlparser", +] + +[[package]] +name = "debugid" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bef552e6f588e446098f6ba40d89ac146c8c7b64aade83c051ee00bb5d2bc18d" +dependencies = [ + "uuid", +] + +[[package]] +name = "derivative" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcc3dd5e9e9c0b295d6e1e4d811fb6f157d5ffd784b8d202fc62eac8035a770b" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "destructure_traitobject" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c877555693c14d2f84191cfd3ad8582790fc52b5e2274b40b59cf5f5cea25c7" + +[[package]] +name = "digest" +version = "0.10.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" +dependencies = [ + "block-buffer", + "crypto-common", + "subtle", +] + +[[package]] +name = "doc-comment" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" + +[[package]] +name = "either" +version = "1.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a26ae43d7bcc3b814de94796a5e736d4029efb0ee900c12e2d54c993ad1a1e07" + +[[package]] +name = "equivalent" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" + +[[package]] +name = "errno" +version = "0.3.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a258e46cdc063eb8519c00b9fc845fc47bcfca4130e2f08e88665ceda8474245" +dependencies = [ + "libc", + "windows-sys", +] + +[[package]] +name = "fastrand" +version = "2.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "25cbce373ec4653f1a01a31e8a5e5ec0c622dc27ff9c4e6606eefef5cbbed4a5" + +[[package]] +name = "findshlibs" +version = "0.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40b9e59cd0f7e0806cca4be089683ecb6434e602038df21fe6bf6711b2f07f64" +dependencies = [ + "cc", + "lazy_static", + "libc", + "winapi", +] + +[[package]] +name = "fixedbitset" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" + +[[package]] +name = "flatbuffers" +version = "23.5.26" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4dac53e22462d78c16d64a1cd22371b54cc3fe94aa15e7886a2fa6e5d1ab8640" +dependencies = [ + "bitflags 1.3.2", + "rustc_version", +] + +[[package]] +name = "flate2" +version = "1.0.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46303f565772937ffe1d394a4fac6f411c6013172fadde9dcdb1e147a086940e" +dependencies = [ + "crc32fast", + "miniz_oxide", +] + +[[package]] +name = "fnv" +version = "1.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" + +[[package]] +name = "form_urlencoded" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e13624c2627564efccf4934284bdd98cbaa14e79b0b5a141218e507b3a823456" +dependencies = [ + "percent-encoding", +] + +[[package]] +name = "futures" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "645c6916888f6cb6350d2550b80fb63e734897a8498abe35cfb732b6487804b0" +dependencies = [ + "futures-channel", + "futures-core", + "futures-executor", + "futures-io", + "futures-sink", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-channel" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eac8f7d7865dcb88bd4373ab671c8cf4508703796caa2b1985a9ca867b3fcb78" +dependencies = [ + "futures-core", + "futures-sink", +] + +[[package]] +name = "futures-core" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dfc6580bb841c5a68e9ef15c77ccc837b40a7504914d52e47b8b0e9bbda25a1d" + +[[package]] +name = "futures-executor" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a576fc72ae164fca6b9db127eaa9a9dda0d61316034f33a0a0d4eda41f02b01d" +dependencies = [ + "futures-core", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-io" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a44623e20b9681a318efdd71c299b6b222ed6f231972bfe2f224ebad6311f0c1" + +[[package]] +name = "futures-macro" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "futures-sink" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9fb8e00e87438d937621c1c6269e53f536c14d3fbd6a042bb24879e57d474fb5" + +[[package]] +name = "futures-task" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38d84fa142264698cdce1a9f9172cf383a0c82de1bddcf3092901442c4097004" + +[[package]] +name = "futures-util" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3d6401deb83407ab3da39eba7e33987a73c3df0c82b4bb5813ee871c19c41d48" +dependencies = [ + "futures-channel", + "futures-core", + "futures-io", + "futures-macro", + "futures-sink", + "futures-task", + "memchr", + "pin-project-lite", + "pin-utils", + "slab", +] + +[[package]] +name = "generic-array" +version = "0.14.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a" +dependencies = [ + "typenum", + "version_check", +] + +[[package]] +name = "getrandom" +version = "0.2.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fe9006bed769170c11f845cf00c7c1e9092aeb3f268e007c3e760ac68008070f" +dependencies = [ + "cfg-if", + "libc", + "wasi", +] + +[[package]] +name = "gimli" +version = "0.28.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4271d37baee1b8c7e4b708028c57d816cf9d2434acb33a549475f78c181f6253" + +[[package]] +name = "glob" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" + +[[package]] +name = "half" +version = "1.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eabb4a44450da02c90444cf74558da904edde8fb4e9035a9a6a4e15445af0bd7" + +[[package]] +name = "half" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ad6a9459c9c30b177b925162351f97e7d967c7ea8bab3b8352805327daf45554" +dependencies = [ + "crunchy", + "num-traits", +] + +[[package]] +name = "hashbrown" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" + +[[package]] +name = "hashbrown" +version = "0.14.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "290f1a1d9242c78d09ce40a5e87e7554ee637af1351968159f4952f028f75604" +dependencies = [ + "ahash", + "allocator-api2", +] + +[[package]] +name = "heck" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d621efb26863f0e9924c6ac577e8275e5e6b77455db64ffa6c65c904e9e132c" +dependencies = [ + "unicode-segmentation", +] + +[[package]] +name = "heck" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" + +[[package]] +name = "hermit-abi" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d77f7ec81a6d05a3abb01ab6eb7590f6083d08449fe5a1c8b1e620283546ccb7" + +[[package]] +name = "hex" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" + +[[package]] +name = "home" +version = "0.5.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3d1354bf6b7235cb4a0576c2619fd4ed18183f689b12b006a0ee7329eeff9a5" +dependencies = [ + "windows-sys", +] + +[[package]] +name = "humantime" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" + +[[package]] +name = "iana-time-zone" +version = "0.1.59" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6a67363e2aa4443928ce15e57ebae94fd8949958fd1223c4cfc0cd473ad7539" +dependencies = [ + "android_system_properties", + "core-foundation-sys", + "iana-time-zone-haiku", + "js-sys", + "wasm-bindgen", + "windows-core", +] + +[[package]] +name = "iana-time-zone-haiku" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f31827a206f56af32e590ba56d5d2d085f558508192593743f16b2306495269f" +dependencies = [ + "cc", +] + +[[package]] +name = "idna" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "634d9b1461af396cad843f47fdba5597a4f9e6ddd4bfb6ff5d85028c25cb12f6" +dependencies = [ + "unicode-bidi", + "unicode-normalization", +] + +[[package]] +name = "indexmap" +version = "1.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" +dependencies = [ + "autocfg", + "hashbrown 0.12.3", +] + +[[package]] +name = "indexmap" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d530e1a18b1cb4c484e6e34556a0d948706958449fca0cab753d649f2bce3d1f" +dependencies = [ + "equivalent", + "hashbrown 0.14.3", +] + +[[package]] +name = "inferno" +version = "0.11.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "321f0f839cd44a4686e9504b0a62b4d69a50b62072144c71c68f5873c167b8d9" +dependencies = [ + "ahash", + "indexmap 2.1.0", + "is-terminal", + "itoa", + "log", + "num-format", + "once_cell", + "quick-xml", + "rgb", + "str_stack", +] + +[[package]] +name = "integer-encoding" +version = "1.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "48dc51180a9b377fd75814d0cc02199c20f8e99433d6762f650d39cdbbd3b56f" + +[[package]] +name = "integer-encoding" +version = "3.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" + +[[package]] +name = "is-terminal" +version = "0.4.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0bad00257d07be169d870ab665980b06cdb366d792ad690bf2e76876dc503455" +dependencies = [ + "hermit-abi", + "rustix", + "windows-sys", +] + +[[package]] +name = "itertools" +version = "0.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +dependencies = [ + "either", +] + +[[package]] +name = "itertools" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b1c173a5686ce8bfa551b3563d0c2170bf24ca44da99c7ca4bfdab5418c3fe57" +dependencies = [ + "either", +] + +[[package]] +name = "itertools" +version = "0.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "25db6b064527c5d482d0423354fcd07a89a2dfe07b67892e62411946db7f07b0" +dependencies = [ + "either", +] + +[[package]] +name = "itoa" +version = "1.0.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b1a46d1a171d865aa5f83f92695765caa047a9b4cbae2cbf37dbd613a793fd4c" + +[[package]] +name = "jni" +version = "0.19.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c6df18c2e3db7e453d3c6ac5b3e9d5182664d28788126d39b91f2d1e22b017ec" +dependencies = [ + "cesu8", + "combine", + "jni-sys", + "log", + "thiserror", + "walkdir", +] + +[[package]] +name = "jni-sys" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" + +[[package]] +name = "jobserver" +version = "0.1.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8c37f63953c4c63420ed5fd3d6d398c719489b9f872b9fa683262f8edd363c7d" +dependencies = [ + "libc", +] + +[[package]] +name = "js-sys" +version = "0.3.66" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cee9c64da59eae3b50095c18d3e74f8b73c0b86d2792824ff01bbce68ba229ca" +dependencies = [ + "wasm-bindgen", +] + +[[package]] +name = "lazy_static" +version = "1.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" + +[[package]] +name = "lexical-core" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2cde5de06e8d4c2faabc400238f9ae1c74d5412d03a7bd067645ccbc47070e46" +dependencies = [ + "lexical-parse-float", + "lexical-parse-integer", + "lexical-util", + "lexical-write-float", + "lexical-write-integer", +] + +[[package]] +name = "lexical-parse-float" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "683b3a5ebd0130b8fb52ba0bdc718cc56815b6a097e28ae5a6997d0ad17dc05f" +dependencies = [ + "lexical-parse-integer", + "lexical-util", + "static_assertions", +] + +[[package]] +name = "lexical-parse-integer" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d0994485ed0c312f6d965766754ea177d07f9c00c9b82a5ee62ed5b47945ee9" +dependencies = [ + "lexical-util", + "static_assertions", +] + +[[package]] +name = "lexical-util" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5255b9ff16ff898710eb9eb63cb39248ea8a5bb036bea8085b1a767ff6c4e3fc" +dependencies = [ + "static_assertions", +] + +[[package]] +name = "lexical-write-float" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "accabaa1c4581f05a3923d1b4cfd124c329352288b7b9da09e766b0668116862" +dependencies = [ + "lexical-util", + "lexical-write-integer", + "static_assertions", +] + +[[package]] +name = "lexical-write-integer" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e1b6f3d1f4422866b68192d62f77bc5c700bee84f3069f2469d7bc8c77852446" +dependencies = [ + "lexical-util", + "static_assertions", +] + +[[package]] +name = "libc" +version = "0.2.151" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "302d7ab3130588088d277783b1e2d2e10c9e9e4a16dd9050e6ec93fb3e7048f4" + +[[package]] +name = "libm" +version = "0.2.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" + +[[package]] +name = "libmimalloc-sys" +version = "0.1.35" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3979b5c37ece694f1f5e51e7ecc871fdb0f517ed04ee45f88d15d6d553cb9664" +dependencies = [ + "cc", + "libc", +] + +[[package]] +name = "linked-hash-map" +version = "0.5.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0717cef1bc8b636c6e1c1bbdefc09e6322da8a9321966e8928ef80d20f7f770f" + +[[package]] +name = "linux-raw-sys" +version = "0.4.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c4cd1a83af159aa67994778be9070f0ae1bd732942279cabb14f86f986a21456" + +[[package]] +name = "lock_api" +version = "0.4.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c168f8615b12bc01f9c17e2eb0cc07dcae1940121185446edc3744920e8ef45" +dependencies = [ + "autocfg", + "scopeguard", +] + +[[package]] +name = "log" +version = "0.4.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b5e6163cb8c49088c2c36f57875e58ccd8c87c7427f7fbd50ea6710b2f3f2e8f" +dependencies = [ + "serde", +] + +[[package]] +name = "log-mdc" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a94d21414c1f4a51209ad204c1776a3d0765002c76c6abcb602a6f09f1e881c7" + +[[package]] +name = "log4rs" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d36ca1786d9e79b8193a68d480a0907b612f109537115c6ff655a3a1967533fd" +dependencies = [ + "anyhow", + "arc-swap", + "chrono", + "derivative", + "fnv", + "humantime", + "libc", + "log", + "log-mdc", + "parking_lot", + "serde", + "serde-value", + "serde_json", + "serde_yaml", + "thiserror", + "thread-id", + "typemap-ors", + "winapi", +] + +[[package]] +name = "lz4" +version = "1.24.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e9e2dd86df36ce760a60f6ff6ad526f7ba1f14ba0356f8254fb6905e6494df1" +dependencies = [ + "libc", + "lz4-sys", +] + +[[package]] +name = "lz4-sys" +version = "1.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57d27b317e207b10f69f5e75494119e391a96f48861ae870d1da6edac98ca900" +dependencies = [ + "cc", + "libc", +] + +[[package]] +name = "md-5" +version = "0.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d89e7ee0cfbedfc4da3340218492196241d89eefb6dab27de5df917a6d2e78cf" +dependencies = [ + "cfg-if", + "digest", +] + +[[package]] +name = "memchr" +version = "2.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "523dc4f511e55ab87b694dc30d0f820d60906ef06413f93d4d7a1385599cc149" + +[[package]] +name = "memmap2" +version = "0.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "45fd3a57831bf88bc63f8cebc0cf956116276e97fef3966103e96416209f7c92" +dependencies = [ + "libc", +] + +[[package]] +name = "mimalloc" +version = "0.1.39" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa01922b5ea280a911e323e4d2fd24b7fe5cc4042e0d2cda3c40775cdc4bdc9c" +dependencies = [ + "libmimalloc-sys", +] + +[[package]] +name = "miniz_oxide" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7" +dependencies = [ + "adler", +] + +[[package]] +name = "multimap" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5ce46fe64a9d73be07dcbe690a38ce1b293be448fd8ce1e6c1b8062c9f72c6a" + +[[package]] +name = "nix" +version = "0.26.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "598beaf3cc6fdd9a5dfb1630c2800c7acd31df7aaf0f565796fba2b53ca1af1b" +dependencies = [ + "bitflags 1.3.2", + "cfg-if", + "libc", +] + +[[package]] +name = "num" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b05180d69e3da0e530ba2a1dae5110317e49e3b7f3d41be227dc5f92e49ee7af" +dependencies = [ + "num-bigint", + "num-complex", + "num-integer", + "num-iter", + "num-rational", + "num-traits", +] + +[[package]] +name = "num-bigint" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "608e7659b5c3d7cba262d894801b9ec9d00de989e8a82bd4bef91d08da45cdc0" +dependencies = [ + "autocfg", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-complex" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ba157ca0885411de85d6ca030ba7e2a83a28636056c7c699b07c8b6f7383214" +dependencies = [ + "num-traits", +] + +[[package]] +name = "num-format" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a652d9771a63711fd3c3deb670acfbe5c30a4072e664d7a3bf5a9e1056ac72c3" +dependencies = [ + "arrayvec", + "itoa", +] + +[[package]] +name = "num-integer" +version = "0.1.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "225d3389fb3509a24c93f5c29eb6bde2586b98d9f016636dff58d7c6f7569cd9" +dependencies = [ + "autocfg", + "num-traits", +] + +[[package]] +name = "num-iter" +version = "0.1.43" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7d03e6c028c5dc5cac6e2dec0efda81fc887605bb3d884578bb6d6bf7514e252" +dependencies = [ + "autocfg", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-rational" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0638a1c9d0a3c0914158145bc76cff373a75a627e6ecbfb71cbe6f453a5a19b0" +dependencies = [ + "autocfg", + "num-bigint", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-traits" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "39e3200413f237f41ab11ad6d161bc7239c84dcb631773ccd7de3dfe4b5c267c" +dependencies = [ + "autocfg", + "libm", +] + +[[package]] +name = "num_cpus" +version = "1.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" +dependencies = [ + "hermit-abi", + "libc", +] + +[[package]] +name = "object" +version = "0.32.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a6a622008b6e321afc04970976f62ee297fdbaa6f95318ca343e3eebb9648441" +dependencies = [ + "memchr", +] + +[[package]] +name = "object_store" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2524735495ea1268be33d200e1ee97455096a0846295a21548cd2f3541de7050" +dependencies = [ + "async-trait", + "bytes", + "chrono", + "futures", + "humantime", + "itertools 0.11.0", + "parking_lot", + "percent-encoding", + "snafu", + "tokio", + "tracing", + "url", + "walkdir", +] + +[[package]] +name = "once_cell" +version = "1.19.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" + +[[package]] +name = "oorandom" +version = "11.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" + +[[package]] +name = "ordered-float" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3305af35278dd29f46fcdd139e0b1fbfae2153f0e5928b39b035542dd31e37b7" +dependencies = [ + "num-traits", +] + +[[package]] +name = "ordered-float" +version = "2.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68f19d67e5a2795c94e73e0bb1cc1a7edeb2e28efd39e2e1c9b7a40c1108b11c" +dependencies = [ + "num-traits", +] + +[[package]] +name = "parking_lot" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3742b2c103b9f06bc9fff0a37ff4912935851bee6d36f3c02bcc755bcfec228f" +dependencies = [ + "lock_api", + "parking_lot_core", +] + +[[package]] +name = "parking_lot_core" +version = "0.9.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c42a9226546d68acdd9c0a280d17ce19bfe27a46bf68784e4066115788d008e" +dependencies = [ + "cfg-if", + "libc", + "redox_syscall", + "smallvec", + "windows-targets 0.48.5", +] + +[[package]] +name = "parquet" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af88740a842787da39b3d69ce5fbf6fce97d20211d3b299fee0a0da6430c74d4" +dependencies = [ + "ahash", + "bytes", + "chrono", + "hashbrown 0.14.3", + "num", + "num-bigint", + "paste", + "seq-macro", + "thrift 0.17.0", + "twox-hash", +] + +[[package]] +name = "parquet-format" +version = "4.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1f0c06cdcd5460967c485f9c40a821746f5955ad81990533c7fae95dbd9bc0b5" +dependencies = [ + "thrift 0.13.0", +] + +[[package]] +name = "parse-zoneinfo" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c705f256449c60da65e11ff6626e0c16a0a0b96aaa348de61376b249bc340f41" +dependencies = [ + "regex", +] + +[[package]] +name = "paste" +version = "1.0.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "de3145af08024dea9fa9914f381a17b8fc6034dfb00f3a84013f7ff43f29ed4c" + +[[package]] +name = "percent-encoding" +version = "2.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" + +[[package]] +name = "petgraph" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e1d3afd2628e69da2be385eb6f2fd57c8ac7977ceeff6dc166ff1657b0e386a9" +dependencies = [ + "fixedbitset", + "indexmap 2.1.0", +] + +[[package]] +name = "phf" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ade2d8b8f33c7333b51bcf0428d37e217e9f32192ae4772156f65063b8ce03dc" +dependencies = [ + "phf_shared", +] + +[[package]] +name = "phf_codegen" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e8d39688d359e6b34654d328e262234662d16cc0f60ec8dcbe5e718709342a5a" +dependencies = [ + "phf_generator", + "phf_shared", +] + +[[package]] +name = "phf_generator" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "48e4cc64c2ad9ebe670cb8fd69dd50ae301650392e81c05f9bfcb2d5bdbc24b0" +dependencies = [ + "phf_shared", + "rand", +] + +[[package]] +name = "phf_shared" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "90fcb95eef784c2ac79119d1dd819e162b5da872ce6f3c3abe1e8ca1c082f72b" +dependencies = [ + "siphasher", +] + +[[package]] +name = "pin-project-lite" +version = "0.2.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8afb450f006bf6385ca15ef45d71d2288452bc3683ce2e2cacc0d18e4be60b58" + +[[package]] +name = "pin-utils" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" + +[[package]] +name = "pkg-config" +version = "0.3.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69d3587f8a9e599cc7ec2c00e331f71c4e69a5f9a4b8a6efd5b07466b9736f9a" + +[[package]] +name = "plotters" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2c224ba00d7cadd4d5c660deaf2098e5e80e07846537c51f9cfa4be50c1fd45" +dependencies = [ + "num-traits", + "plotters-backend", + "plotters-svg", + "wasm-bindgen", + "web-sys", +] + +[[package]] +name = "plotters-backend" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9e76628b4d3a7581389a35d5b6e2139607ad7c75b17aed325f210aa91f4a9609" + +[[package]] +name = "plotters-svg" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38f6d39893cca0701371e3c27294f09797214b86f1fb951b89ade8ec04e2abab" +dependencies = [ + "plotters-backend", +] + +[[package]] +name = "pprof" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ef5c97c51bd34c7e742402e216abdeb44d415fbe6ae41d56b114723e953711cb" +dependencies = [ + "backtrace", + "cfg-if", + "findshlibs", + "inferno", + "libc", + "log", + "nix", + "once_cell", + "parking_lot", + "smallvec", + "symbolic-demangle", + "tempfile", + "thiserror", +] + +[[package]] +name = "ppv-lite86" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" + +[[package]] +name = "proc-macro2" +version = "1.0.75" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "907a61bd0f64c2f29cd1cf1dc34d05176426a3f504a78010f08416ddb7b13708" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "prost" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "444879275cb4fd84958b1a1d5420d15e6fcf7c235fe47f053c9c2a80aceb6001" +dependencies = [ + "bytes", + "prost-derive 0.9.0", +] + +[[package]] +name = "prost" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "146c289cda302b98a28d40c8b3b90498d6e526dd24ac2ecea73e4e491685b94a" +dependencies = [ + "bytes", + "prost-derive 0.12.3", +] + +[[package]] +name = "prost-build" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "62941722fb675d463659e49c4f3fe1fe792ff24fe5bbaa9c08cd3b98a1c354f5" +dependencies = [ + "bytes", + "heck 0.3.3", + "itertools 0.10.5", + "lazy_static", + "log", + "multimap", + "petgraph", + "prost 0.9.0", + "prost-types", + "regex", + "tempfile", + "which", +] + +[[package]] +name = "prost-derive" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9cc1a3263e07e0bf68e96268f37665207b49560d98739662cdfaae215c720fe" +dependencies = [ + "anyhow", + "itertools 0.10.5", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "prost-derive" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "efb6c9a1dd1def8e2124d17e83a20af56f1570d6c2d2bd9e266ccb768df3840e" +dependencies = [ + "anyhow", + "itertools 0.11.0", + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "prost-types" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "534b7a0e836e3c482d2693070f982e39e7611da9695d4d1f5a4b186b51faef0a" +dependencies = [ + "bytes", + "prost 0.9.0", +] + +[[package]] +name = "quick-xml" +version = "0.26.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f50b1c63b38611e7d4d7f68b82d3ad0cc71a2ad2e7f61fc10f1328d917c93cd" +dependencies = [ + "memchr", +] + +[[package]] +name = "quote" +version = "1.0.35" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "291ec9ab5efd934aaf503a6466c5d5251535d108ee747472c3977cc5acc868ef" +dependencies = [ + "proc-macro2", +] + +[[package]] +name = "rand" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" +dependencies = [ + "libc", + "rand_chacha", + "rand_core", +] + +[[package]] +name = "rand_chacha" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" +dependencies = [ + "ppv-lite86", + "rand_core", +] + +[[package]] +name = "rand_core" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" +dependencies = [ + "getrandom", +] + +[[package]] +name = "rayon" +version = "1.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c27db03db7734835b3f53954b534c91069375ce6ccaa2e065441e07d9b6cdb1" +dependencies = [ + "either", + "rayon-core", +] + +[[package]] +name = "rayon-core" +version = "1.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ce3fb6ad83f861aac485e76e1985cd109d9a3713802152be56c3b1f0e0658ed" +dependencies = [ + "crossbeam-deque", + "crossbeam-utils", +] + +[[package]] +name = "redox_syscall" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4722d768eff46b75989dd134e5c353f0d6296e5aaa3132e776cbdb56be7731aa" +dependencies = [ + "bitflags 1.3.2", +] + +[[package]] +name = "regex" +version = "1.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "380b951a9c5e80ddfd6136919eef32310721aa4aacd4889a8d39124b026ab343" +dependencies = [ + "aho-corasick", + "memchr", + "regex-automata", + "regex-syntax", +] + +[[package]] +name = "regex-automata" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f804c7828047e88b2d32e2d7fe5a105da8ee3264f01902f796c8e067dc2483f" +dependencies = [ + "aho-corasick", + "memchr", + "regex-syntax", +] + +[[package]] +name = "regex-syntax" +version = "0.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c08c74e62047bb2de4ff487b251e4a92e24f48745648451635cec7d591162d9f" + +[[package]] +name = "rgb" +version = "0.8.37" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05aaa8004b64fd573fc9d002f4e632d51ad4f026c2b5ba95fcb6c2f32c2c47d8" +dependencies = [ + "bytemuck", +] + +[[package]] +name = "rustc-demangle" +version = "0.1.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" + +[[package]] +name = "rustc_version" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" +dependencies = [ + "semver", +] + +[[package]] +name = "rustix" +version = "0.38.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72e572a5e8ca657d7366229cdde4bd14c4eb5499a9573d4d366fe1b599daa316" +dependencies = [ + "bitflags 2.4.1", + "errno", + "libc", + "linux-raw-sys", + "windows-sys", +] + +[[package]] +name = "rustversion" +version = "1.0.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7ffc183a10b4478d04cbbbfc96d0873219d962dd5accaff2ffbd4ceb7df837f4" + +[[package]] +name = "ryu" +version = "1.0.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f98d2aa92eebf49b69786be48e4477826b256916e84a57ff2a4f21923b48eb4c" + +[[package]] +name = "same-file" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93fc1dc3aaa9bfed95e02e6eadabb4baf7e3078b0bd1b4d7b6b0b68378900502" +dependencies = [ + "winapi-util", +] + +[[package]] +name = "scopeguard" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" + +[[package]] +name = "semver" +version = "1.0.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b97ed7a9823b74f99c7742f5336af7be5ecd3eeafcb1507d1fa93347b1d589b0" + +[[package]] +name = "seq-macro" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" + +[[package]] +name = "serde" +version = "1.0.194" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b114498256798c94a0689e1a15fec6005dee8ac1f41de56404b67afc2a4b773" +dependencies = [ + "serde_derive", +] + +[[package]] +name = "serde-value" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" +dependencies = [ + "ordered-float 2.10.1", + "serde", +] + +[[package]] +name = "serde_derive" +version = "1.0.194" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a3385e45322e8f9931410f01b3031ec534c3947d0e94c18049af4d9f9907d4e0" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "serde_json" +version = "1.0.111" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "176e46fa42316f18edd598015a5166857fc835ec732f5215eac6b7bdbf0a84f4" +dependencies = [ + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "serde_yaml" +version = "0.8.26" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "578a7433b776b56a35785ed5ce9a7e777ac0598aac5a6dd1b4b18a307c7fc71b" +dependencies = [ + "indexmap 1.9.3", + "ryu", + "serde", + "yaml-rust", +] + +[[package]] +name = "sha2" +version = "0.10.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "793db75ad2bcafc3ffa7c68b215fee268f537982cd901d132f89c6343f3a3dc8" +dependencies = [ + "cfg-if", + "cpufeatures", + "digest", +] + +[[package]] +name = "simd-adler32" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d66dc143e6b11c1eddc06d5c423cfc97062865baf299914ab64caa38182078fe" + +[[package]] +name = "siphasher" +version = "0.3.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38b58827f4464d87d377d175e90bf58eb00fd8716ff0a62f80356b5e61555d0d" + +[[package]] +name = "slab" +version = "0.4.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f92a496fb766b417c996b9c5e57daf2f7ad3b0bebe1ccfca4856390e3d3bb67" +dependencies = [ + "autocfg", +] + +[[package]] +name = "smallvec" +version = "1.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4dccd0940a2dcdf68d092b8cbab7dc0ad8fa938bf95787e1b916b0e3d0e8e970" + +[[package]] +name = "snafu" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e4de37ad025c587a29e8f3f5605c00f70b98715ef90b9061a815b9e59e9042d6" +dependencies = [ + "doc-comment", + "snafu-derive", +] + +[[package]] +name = "snafu-derive" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "990079665f075b699031e9c08fd3ab99be5029b96f3b78dc0709e8f77e4efebf" +dependencies = [ + "heck 0.4.1", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "snap" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" + +[[package]] +name = "sqlparser" +version = "0.40.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7c80afe31cdb649e56c0d9bb5503be9166600d68a852c38dd445636d126858e5" +dependencies = [ + "log", + "sqlparser_derive", +] + +[[package]] +name = "sqlparser_derive" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "stable_deref_trait" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" + +[[package]] +name = "static_assertions" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" + +[[package]] +name = "str_stack" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9091b6114800a5f2141aee1d1b9d6ca3592ac062dc5decb3764ec5895a47b4eb" + +[[package]] +name = "strum" +version = "0.25.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "290d54ea6f91c969195bdbcd7442c8c2a2ba87da8bf60a7ee86a235d4bc1e125" +dependencies = [ + "strum_macros", +] + +[[package]] +name = "strum_macros" +version = "0.25.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23dc1fa9ac9c169a78ba62f0b841814b7abae11bdd047b9c58f893439e309ea0" +dependencies = [ + "heck 0.4.1", + "proc-macro2", + "quote", + "rustversion", + "syn 2.0.48", +] + +[[package]] +name = "subtle" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" + +[[package]] +name = "symbolic-common" +version = "12.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1cccfffbc6bb3bb2d3a26cd2077f4d055f6808d266f9d4d158797a4c60510dfe" +dependencies = [ + "debugid", + "memmap2", + "stable_deref_trait", + "uuid", +] + +[[package]] +name = "symbolic-demangle" +version = "12.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "76a99812da4020a67e76c4eb41f08c87364c14170495ff780f30dd519c221a68" +dependencies = [ + "cpp_demangle", + "rustc-demangle", + "symbolic-common", +] + +[[package]] +name = "syn" +version = "1.0.109" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "syn" +version = "2.0.48" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f3531638e407dfc0814761abb7c00a5b54992b849452a0646b7f65c9f770f3f" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "tempfile" +version = "3.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "01ce4141aa927a6d1bd34a041795abd0db1cccba5d5f24b009f694bdf3a1f3fa" +dependencies = [ + "cfg-if", + "fastrand", + "redox_syscall", + "rustix", + "windows-sys", +] + +[[package]] +name = "thiserror" +version = "1.0.56" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d54378c645627613241d077a3a79db965db602882668f9136ac42af9ecb730ad" +dependencies = [ + "thiserror-impl", +] + +[[package]] +name = "thiserror-impl" +version = "1.0.56" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa0faa943b50f3db30a20aa7e265dbc66076993efed8463e8de414e5d06d3471" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "thread-id" +version = "4.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0ec81c46e9eb50deaa257be2f148adf052d1fb7701cfd55ccfab2525280b70b" +dependencies = [ + "libc", + "winapi", +] + +[[package]] +name = "threadpool" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d050e60b33d41c19108b32cea32164033a9013fe3b46cbd4457559bfbf77afaa" +dependencies = [ + "num_cpus", +] + +[[package]] +name = "thrift" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c6d965454947cc7266d22716ebfd07b18d84ebaf35eec558586bbb2a8cb6b5b" +dependencies = [ + "byteorder", + "integer-encoding 1.1.7", + "log", + "ordered-float 1.1.1", + "threadpool", +] + +[[package]] +name = "thrift" +version = "0.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" +dependencies = [ + "byteorder", + "integer-encoding 3.0.4", + "log", + "ordered-float 2.10.1", + "threadpool", +] + +[[package]] +name = "tiny-keccak" +version = "2.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c9d3793400a45f954c52e73d068316d76b6f4e36977e3fcebb13a2721e80237" +dependencies = [ + "crunchy", +] + +[[package]] +name = "tinytemplate" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be4d6b5f19ff7664e8c98d03e2139cb510db9b0a60b55f8e8709b689d939b6bc" +dependencies = [ + "serde", + "serde_json", +] + +[[package]] +name = "tinyvec" +version = "1.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87cc5ceb3875bb20c2890005a4e226a4651264a5c75edb2421b52861a0a0cb50" +dependencies = [ + "tinyvec_macros", +] + +[[package]] +name = "tinyvec_macros" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" + +[[package]] +name = "tokio" +version = "1.35.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c89b4efa943be685f629b149f53829423f8f5531ea21249408e8e2f8671ec104" +dependencies = [ + "backtrace", + "bytes", + "num_cpus", + "parking_lot", + "pin-project-lite", + "tokio-macros", +] + +[[package]] +name = "tokio-macros" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "tokio-stream" +version = "0.1.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "397c988d37662c7dda6d2208364a706264bf3d6138b11d436cbac0ad38832842" +dependencies = [ + "futures-core", + "pin-project-lite", + "tokio", +] + +[[package]] +name = "tokio-util" +version = "0.7.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5419f34732d9eb6ee4c3578b7989078579b7f039cbbb9ca2c4da015749371e15" +dependencies = [ + "bytes", + "futures-core", + "futures-sink", + "pin-project-lite", + "tokio", +] + +[[package]] +name = "tracing" +version = "0.1.40" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c3523ab5a71916ccf420eebdf5521fcef02141234bbc0b8a49f2fdc4544364ef" +dependencies = [ + "pin-project-lite", + "tracing-attributes", + "tracing-core", +] + +[[package]] +name = "tracing-attributes" +version = "0.1.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "tracing-core" +version = "0.1.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c06d3da6113f116aaee68e4d601191614c9053067f9ab7f6edbcb161237daa54" +dependencies = [ + "once_cell", +] + +[[package]] +name = "twox-hash" +version = "1.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" +dependencies = [ + "cfg-if", + "static_assertions", +] + +[[package]] +name = "typemap-ors" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a68c24b707f02dd18f1e4ccceb9d49f2058c2fb86384ef9972592904d7a28867" +dependencies = [ + "unsafe-any-ors", +] + +[[package]] +name = "typenum" +version = "1.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" + +[[package]] +name = "unicode-bidi" +version = "0.3.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6f2528f27a9eb2b21e69c95319b30bd0efd85d09c379741b0f78ea1d86be2416" + +[[package]] +name = "unicode-ident" +version = "1.0.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3354b9ac3fae1ff6755cb6db53683adb661634f67557942dea4facebec0fee4b" + +[[package]] +name = "unicode-normalization" +version = "0.1.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c5713f0fc4b5db668a2ac63cdb7bb4469d8c9fed047b1d0292cc7b0ce2ba921" +dependencies = [ + "tinyvec", +] + +[[package]] +name = "unicode-segmentation" +version = "1.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1dd624098567895118886609431a7c3b8f516e41d30e0643f03d94592a147e36" + +[[package]] +name = "unicode-width" +version = "0.1.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e51733f11c9c4f72aa0c160008246859e340b00807569a0da0e7a1079b27ba85" + +[[package]] +name = "unsafe-any-ors" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e0a303d30665362d9680d7d91d78b23f5f899504d4f08b3c4cf08d055d87c0ad" +dependencies = [ + "destructure_traitobject", +] + +[[package]] +name = "url" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "31e6302e3bb753d46e83516cae55ae196fc0c309407cf11ab35cc51a4c2a4633" +dependencies = [ + "form_urlencoded", + "idna", + "percent-encoding", +] + +[[package]] +name = "uuid" +version = "1.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5e395fcf16a7a3d8127ec99782007af141946b4795001f876d54fb0d55978560" +dependencies = [ + "getrandom", +] + +[[package]] +name = "version_check" +version = "0.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" + +[[package]] +name = "walkdir" +version = "2.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d71d857dc86794ca4c280d616f7da00d2dbfd8cd788846559a6813e6aa4b54ee" +dependencies = [ + "same-file", + "winapi-util", +] + +[[package]] +name = "wasi" +version = "0.11.0+wasi-snapshot-preview1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" + +[[package]] +name = "wasm-bindgen" +version = "0.2.89" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ed0d4f68a3015cc185aff4db9506a015f4b96f95303897bfa23f846db54064e" +dependencies = [ + "cfg-if", + "wasm-bindgen-macro", +] + +[[package]] +name = "wasm-bindgen-backend" +version = "0.2.89" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b56f625e64f3a1084ded111c4d5f477df9f8c92df113852fa5a374dbda78826" +dependencies = [ + "bumpalo", + "log", + "once_cell", + "proc-macro2", + "quote", + "syn 2.0.48", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-macro" +version = "0.2.89" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0162dbf37223cd2afce98f3d0785506dcb8d266223983e4b5b525859e6e182b2" +dependencies = [ + "quote", + "wasm-bindgen-macro-support", +] + +[[package]] +name = "wasm-bindgen-macro-support" +version = "0.2.89" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0eb82fcb7930ae6219a7ecfd55b217f5f0893484b7a13022ebb2b2bf20b5283" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", + "wasm-bindgen-backend", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-shared" +version = "0.2.89" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7ab9b36309365056cd639da3134bf87fa8f3d86008abf99e612384a6eecd459f" + +[[package]] +name = "web-sys" +version = "0.3.66" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "50c24a44ec86bb68fbecd1b3efed7e85ea5621b39b35ef2766b66cd984f8010f" +dependencies = [ + "js-sys", + "wasm-bindgen", +] + +[[package]] +name = "which" +version = "4.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87ba24419a2078cd2b0f2ede2691b6c66d8e47836da3b6db8265ebad47afbfc7" +dependencies = [ + "either", + "home", + "once_cell", + "rustix", +] + +[[package]] +name = "winapi" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" +dependencies = [ + "winapi-i686-pc-windows-gnu", + "winapi-x86_64-pc-windows-gnu", +] + +[[package]] +name = "winapi-i686-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" + +[[package]] +name = "winapi-util" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f29e6f9198ba0d26b4c9f07dbe6f9ed633e1f3d5b8b414090084349e46a52596" +dependencies = [ + "winapi", +] + +[[package]] +name = "winapi-x86_64-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" + +[[package]] +name = "windows-core" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" +dependencies = [ + "windows-targets 0.52.0", +] + +[[package]] +name = "windows-sys" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" +dependencies = [ + "windows-targets 0.52.0", +] + +[[package]] +name = "windows-targets" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a2fa6e2155d7247be68c096456083145c183cbbbc2764150dda45a87197940c" +dependencies = [ + "windows_aarch64_gnullvm 0.48.5", + "windows_aarch64_msvc 0.48.5", + "windows_i686_gnu 0.48.5", + "windows_i686_msvc 0.48.5", + "windows_x86_64_gnu 0.48.5", + "windows_x86_64_gnullvm 0.48.5", + "windows_x86_64_msvc 0.48.5", +] + +[[package]] +name = "windows-targets" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a18201040b24831fbb9e4eb208f8892e1f50a37feb53cc7ff887feb8f50e7cd" +dependencies = [ + "windows_aarch64_gnullvm 0.52.0", + "windows_aarch64_msvc 0.52.0", + "windows_i686_gnu 0.52.0", + "windows_i686_msvc 0.52.0", + "windows_x86_64_gnu 0.52.0", + "windows_x86_64_gnullvm 0.52.0", + "windows_x86_64_msvc 0.52.0", +] + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb7764e35d4db8a7921e09562a0304bf2f93e0a51bfccee0bd0bb0b666b015ea" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbaa0368d4f1d2aaefc55b6fcfee13f41544ddf36801e793edbbfd7d7df075ef" + +[[package]] +name = "windows_i686_gnu" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" + +[[package]] +name = "windows_i686_gnu" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a28637cb1fa3560a16915793afb20081aba2c92ee8af57b4d5f28e4b3e7df313" + +[[package]] +name = "windows_i686_msvc" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" + +[[package]] +name = "windows_i686_msvc" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ffe5e8e31046ce6230cc7215707b816e339ff4d4d67c65dffa206fd0f7aa7b9a" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3d6fa32db2bc4a2f5abeacf2b69f7992cd09dca97498da74a151a3132c26befd" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a657e1e9d3f514745a572a6846d3c7aa7dbe1658c056ed9c3344c4109a6949e" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dff9641d1cd4be8d1a070daf9e3773c5f67e78b4d9d42263020c057706765c04" + +[[package]] +name = "yaml-rust" +version = "0.4.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "56c1936c4cc7a1c9ab21a1ebb602eb942ba868cbd44a99cb7cdc5892335e1c85" +dependencies = [ + "linked-hash-map", +] + +[[package]] +name = "zerocopy" +version = "0.7.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "74d4d3961e53fa4c9a25a8637fc2bfaf2595b3d3ae34875568a5cf64787716be" +dependencies = [ + "zerocopy-derive", +] + +[[package]] +name = "zerocopy-derive" +version = "0.7.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "zstd" +version = "0.11.2+zstd.1.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "20cc960326ece64f010d2d2107537f26dc589a6573a316bd5b1dba685fa5fde4" +dependencies = [ + "zstd-safe", +] + +[[package]] +name = "zstd-safe" +version = "5.0.2+zstd.1.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d2a5585e04f9eea4b2a3d1eca508c4dee9592a89ef6f450c11719da0726f4db" +dependencies = [ + "libc", + "zstd-sys", +] + +[[package]] +name = "zstd-sys" +version = "2.0.9+zstd.1.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9e16efa8a874a0481a574084d34cc26fdb3b99627480f785888deb6386506656" +dependencies = [ + "cc", + "pkg-config", +] diff --git a/core/Cargo.toml b/core/Cargo.toml new file mode 100644 index 000000000..adc3732e3 --- /dev/null +++ b/core/Cargo.toml @@ -0,0 +1,115 @@ +# 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] +name = "comet" +version = "0.1.0" +edition = "2021" +include = [ + "benches/*.rs", + "src/**/*.rs", + "Cargo.toml", +] + +# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html + +[dependencies] +parquet-format = "4.0.0" # This must be kept in sync with that from parquet crate +arrow = { version = "~49.0.0", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow-array = { version = "~49.0.0" } +arrow-data = { version = "~49.0.0" } +arrow-schema = { version = "~49.0.0" } +arrow-string = { version = "~49.0.0" } +parquet = { version = "~49.0.0", default-features = false, features = ["experimental"] } +half = { version = "~2.1", default-features = false } +futures = "0.3.28" +mimalloc = { version = "*", default-features = false, optional = true } +tokio = { version = "1", features = ["rt-multi-thread"] } +tokio-stream = { version = "0.1.8" } +async-trait = "0.1" +log = "0.4" +log4rs = "1.2.0" +thiserror = "1" +serde = { version = "1", features = ["derive"] } +lazy_static = "1.4.0" +prost = "0.12.1" +thrift = "0.17" +jni = "0.19" +byteorder = "1.4.3" +snap = "1.1" +brotli = "3.3" +flate2 = "1.0" +lz4 = "1.24" +zstd = "0.11" +rand = "0.8" +num = "0.4" +bytes = "1.5.0" +hashbrown = { version = "0.14", features = ["raw"] } +parking_lot = "0.12" +tempfile = "3.8.0" +ahash = { version = "0.8", default-features = false } +itertools = "0.11.0" +chrono = { version = "0.4", default-features = false, features = ["clock"] } +chrono-tz = { version = "0.8" } +paste = "1.0.14" +datafusion-common = { version = "34.0.0" } +datafusion = { default-features = false, version = "34.0.0", features = ["unicode_expressions"] } +datafusion-physical-expr = { version = "34.0.0", default-features = false , features = ["unicode_expressions"] } +unicode-segmentation = "^1.10.1" +once_cell = "1.18.0" +regex = "1.9.6" +crc32fast = "1.3.2" +simd-adler32 = "0.3.7" + +[build-dependencies] +prost-build = "0.9.0" + +[dev-dependencies] +pprof = { version = "0.13.0", features = ["flamegraph"] } +criterion = "0.5.1" +jni = { version = "0.19", features = ["invocation"] } +lazy_static = "1.4" +assertables = "7" + +[features] +default = [] +nightly = [] + +[profile.release] +debug = true +overflow-checks = false +lto = "thin" +codegen-units = 1 +strip = "debuginfo" + +[lib] +name = "comet" +# "rlib" is for benchmarking with criterion. +crate_type = ["cdylib", "rlib"] + +[[bench]] +name = "parquet_read" +harness = false + +[[bench]] +name = "bit_util" +harness = false + +[[bench]] +name = "hash" +harness = false + diff --git a/core/benches/bit_util.rs b/core/benches/bit_util.rs new file mode 100644 index 000000000..e92dd6375 --- /dev/null +++ b/core/benches/bit_util.rs @@ -0,0 +1,182 @@ +// 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. + +use std::{mem::size_of, time::Duration}; + +use rand::{thread_rng, Rng}; + +use arrow::buffer::Buffer; +use comet::common::bit::{ + log2, read_num_bytes_u32, read_num_bytes_u64, set_bits, BitReader, BitWriter, +}; +use criterion::{black_box, criterion_group, criterion_main, BenchmarkId, Criterion}; + +/// Benchmark to measure bit_util performance. +/// To run this benchmark: +/// `cd core && cargo bench --bench bit_util` +/// Results will be written to "core/target/criterion/bit_util/" +fn criterion_benchmark(c: &mut Criterion) { + let mut group = c.benchmark_group("bit_util"); + + const N: usize = 1024 * 1024; + let mut writer: BitWriter = BitWriter::new(N * 10); + for _ in 0..N { + if !writer.put_vlq_int(thread_rng().gen::()) { + break; + } + } + let buffer = writer.consume(); + let buffer = Buffer::from(&buffer); + + // log2 + for bits in (0..64).step_by(3) { + let x = 1u64 << bits; + group.bench_with_input(BenchmarkId::new("log2", bits), &x, |b, &x| { + b.iter(|| log2(black_box(x))); + }); + } + + // set_bits + for offset in (0..16).step_by(3) { + for length in (0..16).step_by(3) { + let x = (offset, length); + group.bench_with_input( + BenchmarkId::new("set_bits", format!("offset_{}_length_{}", x.0, x.1)), + &x, + |b, &x| { + b.iter(|| set_bits(&mut [0u8; 4], black_box(x.0), black_box(x.1))); + }, + ); + } + } + + // get_vlq_int + group.bench_function("get_vlq_int", |b| { + b.iter(|| { + let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); + bench_get_vlq_int(&mut reader) + }) + }); + + // get_bits + for offset in (0..32).step_by(17) { + for num_bits in (1..5).step_by(1) { + let x = (offset, num_bits); + group.bench_with_input( + BenchmarkId::new("get_bits", format!("offset_{}_num_bits_{}", x.0, x.1)), + &x, + |b, &x| { + let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); + b.iter(|| reader.get_bits(&mut [0u8; 4], black_box(x.0), black_box(x.1))); + }, + ); + } + } + + // get_aligned + for num_bytes in (1..=size_of::()).step_by(3) { + let x = num_bytes; + group.bench_with_input( + BenchmarkId::new("get_aligned", format!("u8_num_bytes_{}", x)), + &x, + |b, &x| { + let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); + b.iter(|| reader.get_aligned::(black_box(x))); + }, + ); + } + for num_bytes in (1..=size_of::()).step_by(3) { + let x = num_bytes; + group.bench_with_input( + BenchmarkId::new("get_aligned", format!("u32_num_bytes_{}", x)), + &x, + |b, &x| { + let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); + b.iter(|| reader.get_aligned::(black_box(x))); + }, + ); + } + for num_bytes in (1..=size_of::()).step_by(3) { + let x = num_bytes; + group.bench_with_input( + BenchmarkId::new("get_aligned", format!("i32_num_bytes_{}", x)), + &x, + |b, &x| { + let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); + b.iter(|| reader.get_aligned::(black_box(x))); + }, + ); + } + + // get_value + for num_bytes in (1..=size_of::()).step_by(3) { + let x = num_bytes * 8; + group.bench_with_input( + BenchmarkId::new("get_value", format!("i32_num_bits_{}", x)), + &x, + |b, &x| { + let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); + b.iter(|| reader.get_value::(black_box(x))); + }, + ); + } + + // read_num_bytes_u64 + for num_bytes in (1..=8).step_by(7) { + let x = num_bytes; + group.bench_with_input( + BenchmarkId::new("read_num_bytes_u64", format!("num_bytes_{}", x)), + &x, + |b, &x| { + b.iter(|| read_num_bytes_u64(black_box(x), black_box(buffer.as_slice()))); + }, + ); + } + + // read_num_bytes_u32 + for num_bytes in (1..=4).step_by(3) { + let x = num_bytes; + group.bench_with_input( + BenchmarkId::new("read_num_bytes_u32", format!("num_bytes_{}", x)), + &x, + |b, &x| { + b.iter(|| read_num_bytes_u32(black_box(x), black_box(buffer.as_slice()))); + }, + ); + } + + group.finish(); +} + +fn bench_get_vlq_int(reader: &mut BitReader) { + while let Some(v) = reader.get_vlq_int() { + black_box(v); + } +} + +fn config() -> Criterion { + Criterion::default() + .measurement_time(Duration::from_millis(500)) + .warm_up_time(Duration::from_millis(500)) +} + +criterion_group! { + name = benches; + config = config(); + targets = criterion_benchmark +} +criterion_main!(benches); diff --git a/core/benches/common.rs b/core/benches/common.rs new file mode 100644 index 000000000..059721698 --- /dev/null +++ b/core/benches/common.rs @@ -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. + +use arrow::{ + array::{DictionaryArray, Int64Array, PrimitiveArray}, + datatypes::{ArrowPrimitiveType, Int32Type}, +}; +use arrow_schema::ArrowError; +use rand::{ + distributions::{Distribution, Standard}, + rngs::StdRng, + Rng, SeedableRng, +}; +use std::sync::Arc; + +/// Returns fixed seedable RNG +pub fn seedable_rng() -> StdRng { + StdRng::seed_from_u64(42) +} + +pub fn create_int64_array(size: usize, null_density: f32, min: i64, max: i64) -> Int64Array { + let mut rng = seedable_rng(); + (0..size) + .map(|_| { + if rng.gen::() < null_density { + None + } else { + Some(rng.gen_range(min..max)) + } + }) + .collect() +} + +pub fn create_primitive_array(size: usize, null_density: f32) -> PrimitiveArray +where + T: ArrowPrimitiveType, + Standard: Distribution, +{ + let mut rng = seedable_rng(); + (0..size) + .map(|_| { + if rng.gen::() < null_density { + None + } else { + Some(rng.gen()) + } + }) + .collect() +} + +/// Creates a dictionary with random keys and values, with value type `T`. +/// Note here the keys are the dictionary indices. +pub fn create_dictionary_array( + size: usize, + value_size: usize, + null_density: f32, +) -> Result, ArrowError> +where + T: ArrowPrimitiveType, + Standard: Distribution, +{ + // values are not null + let values = create_primitive_array::(value_size, 0.0); + let keys = create_primitive_array::(size, null_density) + .iter() + .map(|v| v.map(|w| w.abs() % (value_size as i32))) + .collect(); + DictionaryArray::try_new(keys, Arc::new(values)) +} diff --git a/core/benches/hash.rs b/core/benches/hash.rs new file mode 100644 index 000000000..dafad79dd --- /dev/null +++ b/core/benches/hash.rs @@ -0,0 +1,109 @@ +// 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. + +#[path = "common.rs"] +mod common; + +use arrow_array::ArrayRef; +use comet::execution::kernels::hash; +use common::*; +use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; +use std::sync::Arc; + +const BATCH_SIZE: usize = 1024 * 8; +const NUM_ITER: usize = 10; +const NULL_FRACTION: f32 = 0.1; + +fn criterion_benchmark(c: &mut Criterion) { + let mut group = c.benchmark_group("hash"); + + let a1: ArrayRef = Arc::new(create_int64_array(BATCH_SIZE, 0.0, 0, BATCH_SIZE as i64)); + let a2: ArrayRef = Arc::new(create_int64_array(BATCH_SIZE, 0.0, 0, BATCH_SIZE as i64)); + let a3: ArrayRef = Arc::new(create_int64_array( + BATCH_SIZE, + NULL_FRACTION, + 0, + BATCH_SIZE as i64, + )); + let a4: ArrayRef = Arc::new(create_int64_array( + BATCH_SIZE, + NULL_FRACTION, + 0, + BATCH_SIZE as i64, + )); + + group.bench_function( + BenchmarkId::new("hash_i64_single_nonnull", BATCH_SIZE), + |b| { + let input = vec![a1.clone()]; + let mut dst = vec![0; BATCH_SIZE]; + + b.iter(|| { + for _ in 0..NUM_ITER { + hash(&input, &mut dst); + } + }); + }, + ); + group.bench_function(BenchmarkId::new("hash_i64_single_null", BATCH_SIZE), |b| { + let input = vec![a3.clone()]; + let mut dst = vec![0; BATCH_SIZE]; + + b.iter(|| { + for _ in 0..NUM_ITER { + hash(&input, &mut dst); + } + }); + }); + group.bench_function( + BenchmarkId::new("hash_i64_multiple_nonnull", BATCH_SIZE), + |b| { + let input = vec![a1.clone(), a2.clone()]; + let mut dst = vec![0; BATCH_SIZE]; + + b.iter(|| { + for _ in 0..NUM_ITER { + hash(&input, &mut dst); + } + }); + }, + ); + group.bench_function( + BenchmarkId::new("hash_i64_multiple_null", BATCH_SIZE), + |b| { + let input = vec![a3.clone(), a4.clone()]; + let mut dst = vec![0; BATCH_SIZE]; + + b.iter(|| { + for _ in 0..NUM_ITER { + hash(&input, &mut dst); + } + }); + }, + ); +} + +fn config() -> Criterion { + Criterion::default() +} + +criterion_group! { + name = benches; + config = config(); + targets = criterion_benchmark +} +criterion_main!(benches); diff --git a/core/benches/parquet_read.rs b/core/benches/parquet_read.rs new file mode 100644 index 000000000..7dcfab7a4 --- /dev/null +++ b/core/benches/parquet_read.rs @@ -0,0 +1,217 @@ +// 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. + +mod perf; + +use std::sync::Arc; + +use arrow::{array::ArrayData, buffer::Buffer}; +use comet::parquet::{read::ColumnReader, util::jni::TypePromotionInfo}; +use criterion::{criterion_group, criterion_main, Criterion}; +use parquet::{ + basic::{Encoding, Type as PhysicalType}, + column::page::{PageIterator, PageReader}, + data_type::Int32Type, + schema::types::{ + ColumnDescPtr, ColumnDescriptor, ColumnPath, PrimitiveTypeBuilder, SchemaDescPtr, TypePtr, + }, +}; + +use comet::parquet::util::test_common::page_util::{ + DataPageBuilder, DataPageBuilderImpl, InMemoryPageIterator, +}; + +use perf::FlamegraphProfiler; +use rand::{prelude::StdRng, Rng, SeedableRng}; + +fn bench(c: &mut Criterion) { + let expected_num_values: usize = NUM_PAGES * VALUES_PER_PAGE; + let mut group = c.benchmark_group("comet_parquet_read"); + let schema = build_test_schema(); + + let pages = build_plain_int32_pages(schema.clone(), schema.column(0), 0.0); + group.bench_function("INT/PLAIN/NOT_NULL", |b| { + let t = TypePtr::new( + PrimitiveTypeBuilder::new("f", PhysicalType::INT32) + .with_length(4) + .build() + .unwrap(), + ); + b.iter(|| { + let cd = ColumnDescriptor::new(t.clone(), 0, 0, ColumnPath::from(Vec::new())); + let promition_info = TypePromotionInfo::new(PhysicalType::INT32, -1); + let mut column_reader = TestColumnReader::new( + cd, + promition_info, + BATCH_SIZE, + pages.clone(), + expected_num_values, + ); + + let mut total = 0; + for batch in column_reader.by_ref() { + total += batch.len(); + ::std::mem::forget(batch); + } + assert_eq!(total, expected_num_values); + }); + }); +} + +fn profiled() -> Criterion { + Criterion::default().with_profiler(FlamegraphProfiler::new(100)) +} + +criterion_group! { + name = benches; + config = profiled(); + targets = bench +} +criterion_main!(benches); + +fn build_test_schema() -> SchemaDescPtr { + use parquet::schema::{parser::parse_message_type, types::SchemaDescriptor}; + let message_type = " + message test_schema { + REQUIRED INT32 c1; + OPTIONAL INT32 c2; + } + "; + parse_message_type(message_type) + .map(|t| Arc::new(SchemaDescriptor::new(Arc::new(t)))) + .unwrap() +} + +fn seedable_rng() -> StdRng { + StdRng::seed_from_u64(42) +} + +// test data params +const NUM_PAGES: usize = 1000; +const VALUES_PER_PAGE: usize = 10_000; +const BATCH_SIZE: usize = 4096; + +fn build_plain_int32_pages( + schema: SchemaDescPtr, + column_desc: ColumnDescPtr, + null_density: f32, +) -> impl PageIterator + Clone { + let max_def_level = column_desc.max_def_level(); + let max_rep_level = column_desc.max_rep_level(); + let rep_levels = vec![0; VALUES_PER_PAGE]; + let mut rng = seedable_rng(); + let mut pages: Vec = Vec::new(); + let mut int32_value = 0; + for _ in 0..NUM_PAGES { + // generate page + let mut values = Vec::with_capacity(VALUES_PER_PAGE); + let mut def_levels = Vec::with_capacity(VALUES_PER_PAGE); + for _ in 0..VALUES_PER_PAGE { + let def_level = if rng.gen::() < null_density { + max_def_level - 1 + } else { + max_def_level + }; + if def_level == max_def_level { + int32_value += 1; + values.push(int32_value); + } + def_levels.push(def_level); + } + let mut page_builder = + DataPageBuilderImpl::new(column_desc.clone(), values.len() as u32, true); + page_builder.add_rep_levels(max_rep_level, &rep_levels); + page_builder.add_def_levels(max_def_level, &def_levels); + page_builder.add_values::(Encoding::PLAIN, &values); + pages.push(page_builder.consume()); + } + + // Since `InMemoryPageReader` is not exposed from parquet crate, here we use + // `InMemoryPageIterator` instead which is a Iter>. + InMemoryPageIterator::new(schema, column_desc, vec![pages]) +} + +struct TestColumnReader { + inner: ColumnReader, + pages: Box, + batch_size: usize, + total_num_values: usize, + total_num_values_read: usize, + first_page_loaded: bool, +} + +impl TestColumnReader { + pub fn new( + cd: ColumnDescriptor, + promotion_info: TypePromotionInfo, + batch_size: usize, + mut page_iter: impl PageIterator + 'static, + total_num_values: usize, + ) -> Self { + let reader = ColumnReader::get(cd, promotion_info, batch_size, false, false); + let first = page_iter.next().unwrap().unwrap(); + Self { + inner: reader, + pages: first, + batch_size, + total_num_values, + total_num_values_read: 0, + first_page_loaded: false, + } + } + + fn load_page(&mut self) { + if let Some(page) = self.pages.get_next_page().unwrap() { + let num_values = page.num_values() as usize; + let buffer = Buffer::from_slice_ref(page.buffer().data()); + self.inner.set_page_v1(num_values, buffer, page.encoding()); + } + } +} + +impl Iterator for TestColumnReader { + type Item = ArrayData; + + fn next(&mut self) -> Option { + if self.total_num_values_read >= self.total_num_values { + return None; + } + + if !self.first_page_loaded { + self.load_page(); + self.first_page_loaded = true; + } + + self.inner.reset_batch(); + let total = ::std::cmp::min( + self.batch_size, + self.total_num_values - self.total_num_values_read, + ); + + let mut left = total; + while left > 0 { + let (num_read, _) = self.inner.read_batch(left, 0); + if num_read < left { + self.load_page(); + } + left -= num_read; + } + self.total_num_values_read += total; + + Some(self.inner.current_batch()) + } +} diff --git a/core/benches/perf.rs b/core/benches/perf.rs new file mode 100644 index 000000000..f92ec0250 --- /dev/null +++ b/core/benches/perf.rs @@ -0,0 +1,61 @@ +// 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. + +use std::{fs::File, os::raw::c_int, path::Path}; + +use criterion::profiler::Profiler; +use pprof::ProfilerGuard; + +/// A custom profiler for criterion which generates flamegraph. +/// +/// Mostly followed this blog post: https://www.jibbow.com/posts/criterion-flamegraphs/ +/// After `cargo bench --bench -- --profile-time=