diff --git a/arrow-data-source/.travis.yml b/arrow-data-source/.travis.yml deleted file mode 100644 index 5c938a101a06..000000000000 --- a/arrow-data-source/.travis.yml +++ /dev/null @@ -1,45 +0,0 @@ -sudo: required -dist: bionic -language: java -jdk: openjdk8 -jobs: - include: - - - name: oap-native-sql - dist: bionic - jdk: - - openjdk8 - before_install: - - echo ${TRAVIS_COMMIT_MESSAGE} - #- if [[ ${TRAVIS_COMMIT_MESSAGE} != \[oap-native-sql\]* ]]; then travis_terminate 0 ; fi ; - - sudo apt-get install cmake - - sudo apt-get install libboost-all-dev - - export | grep JAVA_HOME - install: - - # Download spark 3.0 - - "[ -f spark ] || mkdir spark && cd spark && wget http://archive.apache.org/dist/spark/spark-3.0.0/spark-3.0.0-bin-hadoop2.7.tgz && cd .." - - "tar -xf ./spark/spark-3.0.0-bin-hadoop2.7.tgz" - - "export SPARK_HOME=`pwd`/spark-3.0.0-bin-hadoop2.7" - before_script: - - cd /tmp - - git clone https://github.com/intel-bigdata/arrow.git - - cd arrow && git checkout oap-master && cd cpp - - sed -i "s/\${Python3_EXECUTABLE}/\/opt\/pyenv\/shims\/python3/g" CMakeLists.txt - - mkdir build && cd build - - cmake .. -DARROW_JNI=ON -DARROW_GANDIVA_JAVA=ON -DARROW_GANDIVA=ON -DARROW_PARQUET=ON -DARROW_CSV=ON -DARROW_HDFS=ON -DARROW_FILESYSTEM=ON -DARROW_WITH_SNAPPY=ON -DARROW_JSON=ON -DARROW_DATASET=ON -DARROW_WITH_LZ4=ON && make - - sudo make install - - cd ../../java - - mvn clean install -q -P arrow-jni -am -Darrow.cpp.build.dir=/tmp/arrow/cpp/build/release/ -DskipTests -Dcheckstyle.skip - script: - - cd ${TRAVIS_BUILD_DIR} - - git submodule init - - cd dep/arrow-data-source - - mvn clean -q install -DskipTests - - cd ${TRAVIS_BUILD_DIR}/core - - mvn clean -q package -DskipTests #skip core tests - - # run native sql unit tests - - count=0; while [ $count -le 3 ]; do echo "Elapsed 3 minutes"; sleep 180; let count++; done & # print log each 3 minutes for 3 times to avoid no-log issue - - mvn test -DmembersOnlySuites=org.apache.spark.sql.travis -DfailIfNoTests=false -Dexec.skip=true &> log-file.log # skip cpp build - - echo '#!/bin/bash' > grep.sh - - echo "module_tested=0; module_should_test=1; tests_total=0; while read -r line; do num=\$(echo \"\$line\" | grep -o -E '[0-9]+'); tests_total=\$((tests_total+num)); done <<<\"\$(grep \"Total number of tests run:\" log-file.log)\"; succeed_total=0; while read -r line; do [[ \$line =~ [^0-9]*([0-9]+)\, ]]; num=\${BASH_REMATCH[1]}; succeed_total=\$((succeed_total+num)); let module_tested++; done <<<\"\$(grep \"succeeded\" log-file.log)\"; if test \$tests_total -eq \$succeed_total -a \$module_tested -eq \$module_should_test; then echo \"All unit tests succeed\"; else echo \"Unit tests failed\"; exit 1; fi" >> grep.sh - - bash grep.sh diff --git a/arrow-data-source/README.md b/arrow-data-source/README.md deleted file mode 100644 index 4e71ee5149f1..000000000000 --- a/arrow-data-source/README.md +++ /dev/null @@ -1,231 +0,0 @@ -# Arrow Data Source - -A Spark DataSource implementation for reading files into Arrow compatible columnar vectors. - -## Note - -The development of this library is still in progress. As a result some of the functionality may not be constantly stable for being used in production environments that have not been fully considered due to the limited testing capabilities so far. - -## Build - -### Prerequisite - -There are some requirements before you build the project. -Please make sure you have already installed the software in your system. - -1. GCC 7.0 or higher version -2. java8 OpenJDK -> yum install java-1.8.0-openjdk -3. cmake 3.16 or higher version -4. maven 3.6 or higher version -5. Hadoop 2.7.5 or higher version -6. Spark 3.1.1 or higher version -7. Intel Optimized Arrow 4.0.0 - -### Building by Conda - -If you already have a working Hadoop Spark Cluster, we provide a Conda package which will automatically install dependencies needed by OAP, you can refer to [OAP-Installation-Guide](../docs/OAP-Installation-Guide.md) for more information. Once finished [OAP-Installation-Guide](../docs/OAP-Installation-Guide.md), you can find built `spark-arrow-datasource-standard--jar-with-dependencies.jar` under `$HOME/miniconda2/envs/oapenv/oap_jars`. -Then you can just skip steps below and jump to [Get Started](#get-started). - -### cmake installation - -If you are facing some trouble when installing cmake, please follow below steps to install cmake. - -``` -// installing cmake 3.16.1 -sudo yum install cmake3 - -// If you have an existing cmake, you can use below command to set it as an option within alternatives command -sudo alternatives --install /usr/local/bin/cmake cmake /usr/bin/cmake 10 --slave /usr/local/bin/ctest ctest /usr/bin/ctest --slave /usr/local/bin/cpack cpack /usr/bin/cpack --slave /usr/local/bin/ccmake ccmake /usr/bin/ccmake --family cmake - -// Set cmake3 as an option within alternatives command -sudo alternatives --install /usr/local/bin/cmake cmake /usr/bin/cmake3 20 --slave /usr/local/bin/ctest ctest /usr/bin/ctest3 --slave /usr/local/bin/cpack cpack /usr/bin/cpack3 --slave /usr/local/bin/ccmake ccmake /usr/bin/ccmake3 --family cmake - -// Use alternatives to choose cmake version -sudo alternatives --config cmake -``` - -### maven installation - -If you are facing some trouble when installing maven, please follow below steps to install maven - -``` -// installing maven 3.6.3 -Go to https://maven.apache.org/download.cgi and download the specific version of maven - -// Below command use maven 3.6.3 as an example -wget https://ftp.wayne.edu/apache/maven/maven-3/3.6.3/binaries/apache-maven-3.6.3-bin.tar.gz -tar xzf apache-maven-3.6.3-bin.tar.gz -mkdir /usr/local/maven -mv apache-maven-3.6.3/ /usr/local/maven/ - -// Set maven 3.6.3 as an option within alternatives command -sudo alternatives --install /usr/bin/mvn mvn /usr/local/maven/apache-maven-3.6.3/bin/mvn 1 - -// Use alternatives to choose mvn version -sudo alternatives --config mvn -``` - -### Hadoop Native Library(Default) - -Please make sure you have set up Hadoop directory properly with Hadoop Native Libraries -By default, Apache Arrow would scan `$HADOOP_HOME` and find the native Hadoop library `libhdfs.so`(under `$HADOOP_HOME/lib/native` directory) to be used for Hadoop client. - -You can also use `ARROW_LIBHDFS_DIR` to configure the location of `libhdfs.so` if it is installed in other directory than `$HADOOP_HOME/lib/native` - -If your SPARK and HADOOP are separated in different nodes, please find `libhdfs.so` in your Hadoop cluster and copy it to SPARK cluster, then use one of the above methods to set it properly. - -For more information, please check -Arrow HDFS interface [documentation](https://github.com/apache/arrow/blob/master/cpp/apidoc/HDFS.md) -Hadoop Native Library, please read the official Hadoop website [documentation](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/NativeLibraries.html) - -### Use libhdfs3 library for better performance(Optional) - -For better performance ArrowDataSource reads HDFS files using the third-party library `libhdfs3`. The library must be pre-installed on machines Spark Executor nodes are running on. - -To install the library, use of [Conda](https://docs.conda.io/en/latest/) is recommended. - -``` -// installing libhdfs3 -conda install -c conda-forge libhdfs3 - -// check the installed library file -ll ~/miniconda/envs/$(YOUR_ENV_NAME)/lib/libhdfs3.so -``` - -To set up libhdfs3, there are two different ways: -Option1: Overwrite the soft link for libhdfs.so -To install libhdfs3.so, you have to create a soft link for libhdfs.so in your Hadoop directory(`$HADOOP_HOME/lib/native` by default). - -``` -ln -f -s libhdfs3.so libhdfs.so -``` - -Option2: -Add env variable to the system -``` -export ARROW_LIBHDFS3_DIR="PATH_TO_LIBHDFS3_DIR/" -``` - -Add following Spark configuration options before running the DataSource to make the library to be recognized: -* `spark.executorEnv.ARROW_LIBHDFS3_DIR = "PATH_TO_LIBHDFS3_DIR/"` -* `spark.executorEnv.LD_LIBRARY_PATH = "PATH_TO_LIBHDFS3_DEPENDENCIES_DIR/"` - -Please notes: If you choose to use libhdfs3.so, there are some other dependency libraries you have to installed such as libprotobuf or libcrypto. - -### Build and install IntelĀ® Optimized Arrow with Datasets Java API -You have to use a customized Arrow to support for our datasets Java API. - -``` -// build arrow-cpp -git clone -b arrow-4.0.0-oap https://github.com/oap-project/arrow.git -cd arrow/cpp -mkdir build -cd build -cmake -DARROW_DEPENDENCY_SOURCE=BUNDLED -DARROW_GANDIVA_JAVA=ON -DARROW_GANDIVA=ON -DARROW_PARQUET=ON -DARROW_CSV=ON -DARROW_HDFS=ON -DARROW_BOOST_USE_SHARED=ON -DARROW_JNI=ON -DARROW_DATASET=ON -DARROW_WITH_PROTOBUF=ON -DARROW_WITH_SNAPPY=ON -DARROW_WITH_LZ4=ON -DARROW_FILESYSTEM=ON -DARROW_JSON=ON .. -make - -// build and install arrow jvm library -cd ../../java -mvn clean install -P arrow-jni -am -Darrow.cpp.build.dir=$PATH_TO_ARROW_SOURCE_CODE/arrow/cpp/build/release -``` - -### Build Arrow Data Source Library - -``` -// Download Arrow Data Source Code -git clone -b https://github.com/oap-project/arrow-data-source.git - -// Go to the directory -cd arrow-data-source - -// build -mvn clean -DskipTests package - -// check built jar library -readlink -f standard/target/spark-arrow-datasource-standard--jar-with-dependencies.jar -``` - -### Download Spark 3.1.1 - -Currently ArrowDataSource works on the Spark 3.1.1 version. - -``` -wget http://archive.apache.org/dist/spark/spark-3.1.1/spark-3.1.1-bin-hadoop2.7.tgz -tar -xf ./spark-3.1.1-bin-hadoop2.7.tgz -export SPARK_HOME=`pwd`/spark-3.1.1-bin-hadoop2.7 -``` - -If you are new to Apache Spark, please go though [Spark's official deploying guide](https://spark.apache.org/docs/latest/cluster-overview.html) before getting started with ArrowDataSource. - -## Get started -### Add extra class pathes to Spark - -To enable ArrowDataSource, the previous built jar `spark-arrow-datasource-standard--jar-with-dependencies.jar` should be added to Spark configuration. Typically the options are: - -* `spark.driver.extraClassPath` : Set to load jar file to driver. -* `spark.executor.extraClassPath` : Set to load jar file to executor. -* `jars` : Set to copy jar file to the executors when using yarn cluster mode. -* `spark.executorEnv.ARROW_LIBHDFS3_DIR` : Optional if you are using a custom libhdfs3.so. -* `spark.executorEnv.LD_LIBRARY_PATH` : Optional if you are using a custom libhdfs3.so. - -For Spark Standalone Mode, please set the above value as relative path to the jar file. -For Spark Yarn Cluster Mode, please set the above value as absolute path to the jar file. - -Example to run Spark Shell with ArrowDataSource jar file -``` -${SPARK_HOME}/bin/spark-shell \ - --verbose \ - --master yarn \ - --driver-memory 10G \ - --conf spark.driver.extraClassPath=$PATH_TO_DATASOURCE_DIR/spark-arrow-datasource-standard--jar-with-dependencies.jar \ - --conf spark.executor.extraClassPath=$PATH_TO_DATASOURCE_DIR/spark-arrow-datasource-standard--jar-with-dependencies.jar \ - --conf spark.driver.cores=1 \ - --conf spark.executor.instances=12 \ - --conf spark.executor.cores=6 \ - --conf spark.executor.memory=20G \ - --conf spark.memory.offHeap.size=80G \ - --conf spark.task.cpus=1 \ - --conf spark.locality.wait=0s \ - --conf spark.sql.shuffle.partitions=72 \ - --conf spark.executorEnv.ARROW_LIBHDFS3_DIR="$PATH_TO_LIBHDFS3_DIR/" \ - --conf spark.executorEnv.LD_LIBRARY_PATH="$PATH_TO_LIBHDFS3_DEPENDENCIES_DIR" -``` - -For more information about these options, please read the official Spark [documentation](https://spark.apache.org/docs/latest/configuration.html#runtime-environment). - -### Run a query with ArrowDataSource (Scala) - -```scala -val path = "${PATH_TO_YOUR_PARQUET_FILE}" -val df = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .option(ArrowOptions.KEY_FILESYSTEM, "hdfs") - .format("arrow") - .load(path) -df.createOrReplaceTempView("my_temp_view") -spark.sql("SELECT * FROM my_temp_view LIMIT 10").show(10) -``` -### To validate if ArrowDataSource works properly - -To validate if ArrowDataSource works, you can go to the DAG to check if ArrowScan has been used from the above example query. - -![Image of ArrowDataSource Validation](../docs/image/arrowdatasource_validation.png) - - -## Work together with ParquetDataSource (experimental) - -We provide a customized replacement of Spark's built-in ParquetFileFormat. By so users don't have -to change existing Parquet-based SQL/code and will be able to read Arrow data from Parquet directly. -More importantly, sometimes the feature could be extremely helpful to make ArrowDataSource work correctly -with some 3rd-party storage tools (e.g. [Delta Lake](https://github.com/delta-io/delta)) that are built on top of ParquetDataSource. - -To replace built-in ParquetDataSource, the only thing has to be done is to place compiled jar `spark-arrow-datasource-parquet-.jar` into -Spark's library folder. - -If you'd like to verify that ParquetDataSource is successfully overwritten by the jar, run following code -before executing SQL job: -``` -ServiceLoaderUtil.ensureParquetFileFormatOverwritten(); -``` - -Note the whole feature is currently **experimental** and only DataSource v1 is supported. V2 support is being planned. diff --git a/arrow-data-source/common/pom.xml b/arrow-data-source/common/pom.xml deleted file mode 100644 index 6372f21d1839..000000000000 --- a/arrow-data-source/common/pom.xml +++ /dev/null @@ -1,111 +0,0 @@ - - - - spark-arrow-datasource - com.intel.oap - 1.2.0-snapshot - ../pom.xml - - - 4.0.0 - spark-arrow-datasource-common - - - - org.apache.arrow - ${arrow-memory.artifact} - ${arrow.version} - runtime - - - org.apache.arrow - arrow-dataset - ${arrow.version} - - - io.netty - netty-common - - - io.netty - netty-buffer - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.core - jackson-annotations - - - compile - - - - ${project.basedir}/src/main/scala - ${project.basedir}/src/test/scala - - - org.apache.maven.plugins - maven-source-plugin - - - attach-sources - - jar - - - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.8.0 - - 1.8 - 1.8 - - - - compile - - compile - - - - - - org.codehaus.mojo - build-helper-maven-plugin - - - add-src-1 - generate-sources - - add-source - - - - ${project.basedir}/src/main/java - - - - - - - - - - - org.scala-tools - maven-scala-plugin - - ${scala.version} - - - - - diff --git a/arrow-data-source/common/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/SparkManagedAllocationListener.java b/arrow-data-source/common/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/SparkManagedAllocationListener.java deleted file mode 100644 index abdabcc26dc3..000000000000 --- a/arrow-data-source/common/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/SparkManagedAllocationListener.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.intel.oap.spark.sql.execution.datasources.v2.arrow; - -import org.apache.arrow.memory.AllocationListener; - -public class SparkManagedAllocationListener implements AllocationListener { - - private final NativeSQLMemoryConsumer consumer; - private final NativeSQLMemoryMetrics metrics; - - public SparkManagedAllocationListener(NativeSQLMemoryConsumer consumer, NativeSQLMemoryMetrics metrics) { - this.consumer = consumer; - this.metrics = metrics; - } - - @Override - public void onPreAllocation(long size) { - consumer.acquire(size); - metrics.inc(size); - } - - @Override - public void onRelease(long size) { - consumer.free(size); - metrics.inc(-size); - } -} diff --git a/arrow-data-source/parquet/pom.xml b/arrow-data-source/parquet/pom.xml deleted file mode 100644 index 7e618fc887e5..000000000000 --- a/arrow-data-source/parquet/pom.xml +++ /dev/null @@ -1,27 +0,0 @@ - - - - spark-arrow-datasource - com.intel.oap - 1.2.0-snapshot - - 4.0.0 - - spark-arrow-datasource-parquet - - - ${project.basedir}/src/main/scala - ${project.basedir}/src/test/scala - - - - - com.intel.oap - spark-arrow-datasource-standard - ${project.version} - - - - diff --git a/arrow-data-source/parquet/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/parquet/ServiceLoaderUtil.scala b/arrow-data-source/parquet/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/parquet/ServiceLoaderUtil.scala deleted file mode 100644 index 001b1c26cf61..000000000000 --- a/arrow-data-source/parquet/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/parquet/ServiceLoaderUtil.scala +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.intel.oap.spark.sql.execution.datasources.v2.parquet - -import java.util.Objects - -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetFileFormatIndicator} - -object ServiceLoaderUtil { - def ensureParquetFileFormatOverwritten(): Unit = { - val fmt = new ParquetFileFormat() - if (!Objects.equals(fmt.toString(), ParquetFileFormatIndicator.OVERWRITTEN_INDICATOR)) { - throw new ClassNotFoundException("ParquetFileFormat is not overwritten by Arrow. Consider " + - "reordering jar dependencies to let the overwritten version to be recognized by JVM") - } - } -} diff --git a/arrow-data-source/parquet/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/arrow-data-source/parquet/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala deleted file mode 100644 index 836e9f2c92a8..000000000000 --- a/arrow-data-source/parquet/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ /dev/null @@ -1,479 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.parquet - -import java.io.IOException -import java.net.URI - -import scala.collection.JavaConverters._ -import scala.util.{Failure, Try} - -import com.intel.oap.spark.sql.execution.datasources.arrow.ArrowFileFormat -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.hadoop.mapreduce.{Job, JobID, OutputCommitter, TaskAttemptContext, TaskAttemptID, TaskID, TaskType} -import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl -import org.apache.parquet.filter2.compat.FilterCompat -import org.apache.parquet.filter2.predicate.FilterApi -import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS -import org.apache.parquet.hadoop.{Footer, ParquetFileReader, ParquetInputFormat, ParquetOutputCommitter, ParquetOutputFormat, ParquetRecordReader} -import org.apache.parquet.hadoop.ParquetOutputFormat.JobSummaryLevel -import org.apache.parquet.hadoop.codec.CodecConfig -import org.apache.parquet.hadoop.util.ContextUtil -import org.apache.parquet.schema.MessageType - -import org.apache.spark.TaskContext -import org.apache.spark.internal.Logging -import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.JoinedRow -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.catalyst.parser.LegacyTypeStringParser -import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.datasources.{DataSourceUtils, OutputWriter, OutputWriterFactory, PartitionedFile, RecordReaderIterator, SchemaMergeUtils} -import org.apache.spark.sql.execution.datasources.parquet.ParquetSQLConf._ -import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.{DataSourceRegister, Filter} -import org.apache.spark.sql.types.{ArrayType, AtomicType, DataType, MapType, StructType, UserDefinedType} -import org.apache.spark.util.{SerializableConfiguration, ThreadUtils} - -/** - * This is expected to overwrite built-in ParquetFileFormat. Read is redirected to ArrowFileFormat's - * read. - */ -class ParquetFileFormat - extends ArrowFileFormat - with DataSourceRegister - with Logging - with Serializable { - - override def shortName(): String = "parquet" - - override def toString: String = ParquetFileFormatIndicator.OVERWRITTEN_INDICATOR - - override def hashCode(): Int = getClass.hashCode() - - override def equals(other: Any): Boolean = other.isInstanceOf[ParquetFileFormat] - - /** - * We copy following codes from Spark's built-in ParquetFileFormat. It's not suggested to - * change any of the logic to make sure we are on the same boat with Spark when it comes - * to Parquet write. - */ - override def prepareWrite(sparkSession: SparkSession, - job: Job, - options: Map[String, String], - dataSchema: StructType): OutputWriterFactory = { - val parquetOptions = new ParquetOptions(options, sparkSession.sessionState.conf) - - val conf = ContextUtil.getConfiguration(job) - - val committerClass = - conf.getClass( - SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key, - classOf[ParquetOutputCommitter], - classOf[OutputCommitter]) - - if (conf.get(SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key) == null) { - logInfo("Using default output committer for Parquet: " + - classOf[ParquetOutputCommitter].getCanonicalName) - } else { - logInfo("Using user defined output committer for Parquet: " + committerClass.getCanonicalName) - } - - conf.setClass( - SQLConf.OUTPUT_COMMITTER_CLASS.key, - committerClass, - classOf[OutputCommitter]) - - // We're not really using `ParquetOutputFormat[Row]` for writing data here, because we override - // it in `ParquetOutputWriter` to support appending and dynamic partitioning. The reason why - // we set it here is to setup the output committer class to `ParquetOutputCommitter`, which is - // bundled with `ParquetOutputFormat[Row]`. - job.setOutputFormatClass(classOf[ParquetOutputFormat[Row]]) - - ParquetOutputFormat.setWriteSupportClass(job, classOf[ParquetWriteSupport]) - - // This metadata is useful for keeping UDTs like Vector/Matrix. - ParquetWriteSupport.setSchema(dataSchema, conf) - - // Sets flags for `ParquetWriteSupport`, which converts Catalyst schema to Parquet - // schema and writes actual rows to Parquet files. - conf.set( - SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key, - sparkSession.sessionState.conf.writeLegacyParquetFormat.toString) - - conf.set( - SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key, - sparkSession.sessionState.conf.parquetOutputTimestampType.toString) - - // Sets compression scheme - conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodecClassName) - - // SPARK-15719: Disables writing Parquet summary files by default. - if (conf.get(ParquetOutputFormat.JOB_SUMMARY_LEVEL) == null - && conf.get(ParquetOutputFormat.ENABLE_JOB_SUMMARY) == null) { - conf.setEnum(ParquetOutputFormat.JOB_SUMMARY_LEVEL, JobSummaryLevel.NONE) - } - - if (ParquetOutputFormat.getJobSummaryLevel(conf) != JobSummaryLevel.NONE - && !classOf[ParquetOutputCommitter].isAssignableFrom(committerClass)) { - // output summary is requested, but the class is not a Parquet Committer - logWarning(s"Committer $committerClass is not a ParquetOutputCommitter and cannot" + - s" create job summaries. " + - s"Set Parquet option ${ParquetOutputFormat.JOB_SUMMARY_LEVEL} to NONE.") - } - - new OutputWriterFactory { - // This OutputWriterFactory instance is deserialized when writing Parquet files on the - // executor side without constructing or deserializing ParquetFileFormat. Therefore, we hold - // another reference to ParquetLogRedirector.INSTANCE here to ensure the latter class is - // initialized. - private val parquetLogRedirector = ParquetLogRedirector.INSTANCE - - override def newInstance( - path: String, - dataSchema: StructType, - context: TaskAttemptContext): OutputWriter = { - new ParquetOutputWriter(path, context) - } - - override def getFileExtension(context: TaskAttemptContext): String = { - CodecConfig.from(context).getCodec.getExtension + ".parquet" - } - } - } - - override def inferSchema(sparkSession: SparkSession, - parameters: Map[String, String], - files: Seq[FileStatus]): Option[StructType] = { - val overwrite = sparkSession.sqlContext.conf.overwriteParquetDataSourceRead - if (overwrite) { - return super.inferSchema(sparkSession, parameters, files) - } - ParquetUtils.inferSchema(sparkSession, parameters, files) - } - - /** - * Returns whether the reader will return the rows as batch or not. - */ - override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { - val conf = sparkSession.sessionState.conf - val overwrite = sparkSession.sqlContext.conf.overwriteParquetDataSourceRead - if (overwrite) { - return super.supportBatch(sparkSession, schema) - } - conf.parquetVectorizedReaderEnabled && conf.wholeStageEnabled && - schema.length <= conf.wholeStageMaxNumFields && - schema.forall(_.dataType.isInstanceOf[AtomicType]) - } - - override def vectorTypes(requiredSchema: StructType, - partitionSchema: StructType, - sqlConf: SQLConf): Option[Seq[String]] = { - val overwrite = sqlConf.overwriteParquetDataSourceRead - if (overwrite) { - return super.vectorTypes(requiredSchema, partitionSchema, sqlConf) - } - Option(Seq.fill(requiredSchema.fields.length + partitionSchema.fields.length)( - if (!sqlConf.offHeapColumnVectorEnabled) { - classOf[OnHeapColumnVector].getName - } else { - classOf[OffHeapColumnVector].getName - } - )) - } - - override def isSplitable(sparkSession: SparkSession, - options: Map[String, String], - path: Path): Boolean = { - val overwrite = sparkSession.sqlContext.conf.overwriteParquetDataSourceRead - if (overwrite) { - return super.isSplitable(sparkSession, options, path) - } - true - } - - override def buildReaderWithPartitionValues(sparkSession: SparkSession, - dataSchema: StructType, - partitionSchema: StructType, - requiredSchema: StructType, - filters: Seq[Filter], - options: Map[String, String], - hadoopConf: Configuration): - (PartitionedFile) => Iterator[InternalRow] = { - val overwrite = sparkSession.sqlContext.conf.overwriteParquetDataSourceRead - if (overwrite) { - return super.buildReaderWithPartitionValues(sparkSession, dataSchema, - partitionSchema, requiredSchema, filters, options, hadoopConf) - } - hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) - hadoopConf.set( - ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, - requiredSchema.json) - hadoopConf.set( - ParquetWriteSupport.SPARK_ROW_SCHEMA, - requiredSchema.json) - hadoopConf.set( - SQLConf.SESSION_LOCAL_TIMEZONE.key, - sparkSession.sessionState.conf.sessionLocalTimeZone) - hadoopConf.setBoolean( - SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, - sparkSession.sessionState.conf.nestedSchemaPruningEnabled) - hadoopConf.setBoolean( - SQLConf.CASE_SENSITIVE.key, - sparkSession.sessionState.conf.caseSensitiveAnalysis) - - ParquetWriteSupport.setSchema(requiredSchema, hadoopConf) - - // Sets flags for `ParquetToSparkSchemaConverter` - hadoopConf.setBoolean( - SQLConf.PARQUET_BINARY_AS_STRING.key, - sparkSession.sessionState.conf.isParquetBinaryAsString) - hadoopConf.setBoolean( - SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, - sparkSession.sessionState.conf.isParquetINT96AsTimestamp) - - val broadcastedHadoopConf = - sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) - - // TODO: if you move this into the closure it reverts to the default values. - // If true, enable using the custom RecordReader for parquet. This only works for - // a subset of the types (no complex types). - val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields) - val sqlConf = sparkSession.sessionState.conf - val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled - val enableVectorizedReader: Boolean = - sqlConf.parquetVectorizedReaderEnabled && - resultSchema.forall(_.dataType.isInstanceOf[AtomicType]) - val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled - val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion - val capacity = sqlConf.parquetVectorizedReaderBatchSize - val enableParquetFilterPushDown: Boolean = sqlConf.parquetFilterPushDown - // Whole stage codegen (PhysicalRDD) is able to deal with batches directly - val returningBatch = supportBatch(sparkSession, resultSchema) - val pushDownDate = sqlConf.parquetFilterPushDownDate - val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp - val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal - val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith - val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold - val isCaseSensitive = sqlConf.caseSensitiveAnalysis - - (file: PartitionedFile) => { - assert(file.partitionValues.numFields == partitionSchema.size) - - val filePath = new Path(new URI(file.filePath)) - val split = - new org.apache.parquet.hadoop.ParquetInputSplit( - filePath, - file.start, - file.start + file.length, - file.length, - Array.empty, - null) - - val sharedConf = broadcastedHadoopConf.value.value - - lazy val footerFileMetaData = - ParquetFileReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData - // Try to push down filters when filter push-down is enabled. - val pushed = if (enableParquetFilterPushDown) { - val parquetSchema = footerFileMetaData.getSchema - val parquetFilters = new ParquetFilters(parquetSchema, pushDownDate, pushDownTimestamp, - pushDownDecimal, pushDownStringStartWith, pushDownInFilterThreshold, isCaseSensitive) - filters - // Collects all converted Parquet filter predicates. Notice that not all predicates can be - // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` - // is used here. - .flatMap(parquetFilters.createFilter(_)) - .reduceOption(FilterApi.and) - } else { - None - } - - // PARQUET_INT96_TIMESTAMP_CONVERSION says to apply timezone conversions to int96 timestamps' - // *only* if the file was created by something other than "parquet-mr", so check the actual - // writer here for this file. We have to do this per-file, as each file in the table may - // have different writers. - // Define isCreatedByParquetMr as function to avoid unnecessary parquet footer reads. - def isCreatedByParquetMr: Boolean = - footerFileMetaData.getCreatedBy().startsWith("parquet-mr") - - val convertTz = - if (timestampConversion && !isCreatedByParquetMr) { - Some(DateTimeUtils.getZoneId(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key))) - } else { - None - } - - val datetimeRebaseMode = DataSourceUtils.datetimeRebaseMode( - footerFileMetaData.getKeyValueMetaData.get, - SQLConf.get.getConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ)) - - val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) - val hadoopAttemptContext = - new TaskAttemptContextImpl(broadcastedHadoopConf.value.value, attemptId) - - // Try to push down filters when filter push-down is enabled. - // Notice: This push-down is RowGroups level, not individual records. - if (pushed.isDefined) { - ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get) - } - val taskContext = Option(TaskContext.get()) - if (enableVectorizedReader) { - val vectorizedReader = new VectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseMode.toString, - "", - enableOffHeapColumnVector && taskContext.isDefined, - capacity) - val iter = new RecordReaderIterator(vectorizedReader) - // SPARK-23457 Register a task completion listener before `initialization`. - taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) - vectorizedReader.initialize(split, hadoopAttemptContext) - logDebug(s"Appending $partitionSchema ${file.partitionValues}") - vectorizedReader.initBatch(partitionSchema, file.partitionValues) - if (returningBatch) { - vectorizedReader.enableReturningBatches() - } - - // UnsafeRowParquetRecordReader appends the columns internally to avoid another copy. - iter.asInstanceOf[Iterator[InternalRow]] - } else { - logDebug(s"Falling back to parquet-mr") - // ParquetRecordReader returns InternalRow - val readSupport = new ParquetReadSupport( - convertTz, enableVectorizedReader = false, datetimeRebaseMode, SQLConf.LegacyBehaviorPolicy.LEGACY) - val reader = if (pushed.isDefined && enableRecordFilter) { - val parquetFilter = FilterCompat.get(pushed.get, null) - new ParquetRecordReader[InternalRow](readSupport, parquetFilter) - } else { - new ParquetRecordReader[InternalRow](readSupport) - } - val iter = new RecordReaderIterator[InternalRow](reader) - // SPARK-23457 Register a task completion listener before `initialization`. - taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) - reader.initialize(split, hadoopAttemptContext) - - val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) - - if (partitionSchema.length == 0) { - // There is no partition columns - iter.map(unsafeProjection) - } else { - val joinedRow = new JoinedRow() - iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues))) - } - } - } - } - - override def supportDataType(dataType: DataType): Boolean = dataType match { - case _: AtomicType => true - - case st: StructType => st.forall { f => supportDataType(f.dataType) } - - case ArrayType(elementType, _) => supportDataType(elementType) - - case MapType(keyType, valueType, _) => - supportDataType(keyType) && supportDataType(valueType) - - case udt: UserDefinedType[_] => supportDataType(udt.sqlType) - - case _ => false - } -} - -object ParquetFileFormat extends Logging { - /** - * Reads Spark SQL schema from a Parquet footer. If a valid serialized Spark SQL schema string - * can be found in the file metadata, returns the deserialized [[StructType]], otherwise, returns - * a [[StructType]] converted from the [[MessageType]] stored in this footer. - */ - def readSchemaFromFooter(footer: Footer, converter: ParquetToSparkSchemaConverter): StructType = { - val fileMetaData = footer.getParquetMetadata.getFileMetaData - fileMetaData - .getKeyValueMetaData - .asScala.toMap - .get(ParquetReadSupport.SPARK_METADATA_KEY) - .flatMap(deserializeSchemaString) - .getOrElse(converter.convert(fileMetaData.getSchema)) - } - - private def deserializeSchemaString(schemaString: String): Option[StructType] = { - // Tries to deserialize the schema string as JSON first, then falls back to the case class - // string parser (data generated by older versions of Spark SQL uses this format). - Try(DataType.fromJson(schemaString).asInstanceOf[StructType]).recover { - case _: Throwable => - logInfo( - "Serialized Spark schema in Parquet key-value metadata is not in JSON format, " + - "falling back to the deprecated DataType.fromCaseClassString parser.") - LegacyTypeStringParser.parseString(schemaString).asInstanceOf[StructType] - }.recoverWith { - case cause: Throwable => - logWarning( - "Failed to parse and ignored serialized Spark schema in " + - s"Parquet key-value metadata:\n\t$schemaString", cause) - Failure(cause) - }.toOption - } - - def mergeSchemasInParallel( - filesToTouch: Seq[FileStatus], - sparkSession: SparkSession): Option[StructType] = { - val assumeBinaryIsString = sparkSession.sessionState.conf.isParquetBinaryAsString - val assumeInt96IsTimestamp = sparkSession.sessionState.conf.isParquetINT96AsTimestamp - - val reader = (files: Seq[FileStatus], conf: Configuration, ignoreCorruptFiles: Boolean) => { - // Converter used to convert Parquet `MessageType` to Spark SQL `StructType` - val converter = new ParquetToSparkSchemaConverter( - assumeBinaryIsString = assumeBinaryIsString, - assumeInt96IsTimestamp = assumeInt96IsTimestamp) - - readParquetFootersInParallel(conf, files, ignoreCorruptFiles) - .map(ParquetFileFormat.readSchemaFromFooter(_, converter)) - } - - SchemaMergeUtils.mergeSchemasInParallel(sparkSession, null, filesToTouch, reader) - } - - private[parquet] def readParquetFootersInParallel( - conf: Configuration, - partFiles: Seq[FileStatus], - ignoreCorruptFiles: Boolean): Seq[Footer] = { - ThreadUtils.parmap(partFiles, "readingParquetFooters", 8) { currentFile => - try { - // Skips row group information since we only need the schema. - // ParquetFileReader.readFooter throws RuntimeException, instead of IOException, - // when it can't read the footer. - Some(new Footer(currentFile.getPath(), - ParquetFileReader.readFooter( - conf, currentFile, SKIP_ROW_GROUPS))) - } catch { case e: RuntimeException => - if (ignoreCorruptFiles) { - logWarning(s"Skipped the footer in the corrupted file: $currentFile", e) - None - } else { - throw new IOException(s"Could not read footer for file: $currentFile", e) - } - } - }.flatten - } -} diff --git a/arrow-data-source/parquet/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatIndicator.scala b/arrow-data-source/parquet/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatIndicator.scala deleted file mode 100644 index 29aa38069b46..000000000000 --- a/arrow-data-source/parquet/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatIndicator.scala +++ /dev/null @@ -1,22 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.parquet - -object ParquetFileFormatIndicator { - val OVERWRITTEN_INDICATOR = "Parquet-Overwritten-By-Arrow" -} diff --git a/arrow-data-source/parquet/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSQLConf.scala b/arrow-data-source/parquet/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSQLConf.scala deleted file mode 100644 index 9e3096189afa..000000000000 --- a/arrow-data-source/parquet/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSQLConf.scala +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.parquet - -import org.apache.spark.sql.internal.SQLConf - -object ParquetSQLConf { - // We default this option value to TRUE. This is because once the code is executed, the compiled - // arrow-datasource-parquet.jar file is supposed to be placed into Spark's lib folder. Which - // means it's user's intention to use the replaced ParquetDataSource. - val OVERWRITE_PARQUET_DATASOURCE_READ = - SQLConf.buildConf("spark.sql.arrow.overwrite.parquet.read") - .doc("Overwrite Parquet datasource v1 with reader of Arrow datasource.") - .booleanConf - .createWithDefault(true) - - implicit def fromSQLConf(c: SQLConf): ParquetSQLConf = { - new ParquetSQLConf(c) - } -} - -class ParquetSQLConf(c: SQLConf) { - def overwriteParquetDataSourceRead: Boolean = - c.getConf(ParquetSQLConf.OVERWRITE_PARQUET_DATASOURCE_READ) -} diff --git a/arrow-data-source/parquet/src/test/scala/com/intel/oap/spark/sql/execution/datasources/v2/parquet/ParquetFileFormatTest.scala b/arrow-data-source/parquet/src/test/scala/com/intel/oap/spark/sql/execution/datasources/v2/parquet/ParquetFileFormatTest.scala deleted file mode 100644 index 84e0fc410a05..000000000000 --- a/arrow-data-source/parquet/src/test/scala/com/intel/oap/spark/sql/execution/datasources/v2/parquet/ParquetFileFormatTest.scala +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.intel.oap.spark.sql.execution.datasources.v2.parquet - -import java.io.File - -import com.intel.oap.vectorized.ArrowWritableColumnVector - -import org.apache.spark.SparkConf -import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.execution.FileSourceScanExec -import org.apache.spark.sql.execution.datasources.parquet.ParquetSQLConf -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils -import org.apache.spark.sql.test.SharedSparkSession - -class ParquetFileFormatTest extends QueryTest with SharedSparkSession { - - private val parquetFile1 = "parquet-1.parquet" - - override protected def sparkConf: SparkConf = { - val conf = super.sparkConf - conf.set("spark.memory.offHeap.size", String.valueOf(256 * 1024 * 1024)) - conf - } - - def closeAllocators(): Unit = { - SparkMemoryUtils.contextAllocator().close() - } - - test("overwrite write only") { - import testImplicits._ - withSQLConf(ParquetSQLConf.OVERWRITE_PARQUET_DATASOURCE_READ.key -> "false") { - ServiceLoaderUtil.ensureParquetFileFormatOverwritten() - spark.read - .json(Seq("{\"col\": -1}", "{\"col\": 0}", "{\"col\": 1}", "{\"col\": 2}", - "{\"col\": null}") - .toDS()) - .repartition(1) - .write - .mode("overwrite") - .parquet(ParquetFileFormatTest.locateResourcePath(parquetFile1)) - val path = ParquetFileFormatTest.locateResourcePath(parquetFile1) - val frame = spark.read.parquet(path) - val eplan = frame.queryExecution.executedPlan - assert(eplan.toString - .contains("Format: Parquet-Overwritten-By-Arrow")) - val scan = eplan.find(_.isInstanceOf[FileSourceScanExec]).get - val typeAssertions = scan.executeColumnar() - .flatMap(b => (0 until b.numCols()).map(b.column(_))) - .map(!_.isInstanceOf[ArrowWritableColumnVector]) - .collect() - assert(typeAssertions.forall(p => p)) - } - } - - test("overwrite read and write") { - import testImplicits._ - ServiceLoaderUtil.ensureParquetFileFormatOverwritten() - spark.read - .json(Seq("{\"col\": -1}", "{\"col\": 0}", "{\"col\": 1}", "{\"col\": 2}", "{\"col\": null}") - .toDS()) - .repartition(1) - .write - .mode("overwrite") - .parquet(ParquetFileFormatTest.locateResourcePath(parquetFile1)) - val path = ParquetFileFormatTest.locateResourcePath(parquetFile1) - val frame = spark.read.parquet(path) - val eplan = frame.queryExecution.executedPlan - assert(eplan.toString - .contains("Format: Parquet-Overwritten-By-Arrow")) - val scan = eplan.find(_.isInstanceOf[FileSourceScanExec]).get - val typeAssertions = scan.executeColumnar() - .flatMap(b => (0 until b.numCols()).map(b.column(_))) - .map(_.isInstanceOf[ArrowWritableColumnVector]) - .collect() - assert(typeAssertions.forall(p => p)) - } -} - -object ParquetFileFormatTest { - private def locateResourcePath(resource: String): String = { - classOf[ParquetFileFormatTest].getClassLoader.getResource("") - .getPath.concat(File.separator).concat(resource) - } -} diff --git a/arrow-data-source/pom.xml b/arrow-data-source/pom.xml deleted file mode 100644 index 3f72871f2dec..000000000000 --- a/arrow-data-source/pom.xml +++ /dev/null @@ -1,244 +0,0 @@ - - - com.intel.oap - native-sql-engine-parent - 1.2.0-snapshot - - - 4.0.0 - com.intel.oap - spark-arrow-datasource - OAP Project Spark Arrow Datasource - pom - 1.2.0-snapshot - 2008 - - common - standard - parquet - - - ${arrow.script.dir} - ${cpp_tests} - ${build_arrow} - ${static_arrow} - ${arrow_root} - - - - - scala-tools.org - Scala-Tools Maven2 Repository - http://scala-tools.org/repo-releases - - - - - - scala-tools.org - Scala-Tools Maven2 Repository - http://scala-tools.org/repo-releases - - - - - - javax.servlet - javax.servlet-api - 3.1.0 - - - org.apache.hadoop - hadoop-client - ${hadoop.version} - provided - - - org.apache.hadoop - hadoop-aws - ${hadoop.version} - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.core - jackson-annotations - - - com.fasterxml.jackson.core - jackson-databind - - - javax.servlet - servlet-api - - - com.sun.jersey - jersey-core - - - com.sun.jersey - jersey-json - - - com.sun.jersey - jersey-server - - - commons-httpclient - commons-httpcore - - - org.slf4j - slf4j-log4j12 - - - log4j - log4j - - - - - org.apache.httpcomponents - httpcore - 4.2 - - - org.scala-lang - scala-library - ${scala.version} - provided - - - junit - junit - 4.4 - test - - - org.apache.spark - spark-sql_${scala.binary.version} - provided - - - org.apache.spark - spark-core_${scala.binary.version} - test-jar - test - - - org.apache.spark - spark-catalyst_${scala.binary.version} - test-jar - test - - - org.apache.spark - spark-sql_${scala.binary.version} - test-jar - test - - - org.scalatest - scalatest_${scala.binary.version} - test - - - - - - - exec-maven-plugin - org.codehaus.mojo - 1.6.0 - false - - - Build arrow - generate-resources - - exec - - - bash - - ${script.dir}/build_arrow.sh - --tests=${datasource.cpp_tests} - --build_arrow=${datasource.build_arrow} - --static_arrow=${datasource.static_arrow} - --arrow_root=${datasource.arrow_root} - - - - - - - maven-clean-plugin - - - - ${script.dir}/build - - - - - - org.scalatest - scalatest-maven-plugin - - - - test - - - - - - org.scala-tools - maven-scala-plugin - - - - compile - testCompile - - process-sources - - - - ${scala.version} - - -target:jvm-1.5 - - - - - org.scalastyle - scalastyle-maven-plugin - 1.0.0 - - true - false - true - false - false - ${project.basedir}/src/main/scala - ${project.basedir}/src/test/scala - ${user.dir}/scalastyle-config.xml - ${project.basedir}/target/scalastyle-output.xml - ${project.build.sourceEncoding} - ${project.reporting.outputEncoding} - - - - - check - - - - - - - diff --git a/arrow-data-source/resource/arrowdatasource_validation.png b/arrow-data-source/resource/arrowdatasource_validation.png deleted file mode 100644 index c7583511a44a..000000000000 Binary files a/arrow-data-source/resource/arrowdatasource_validation.png and /dev/null differ diff --git a/arrow-data-source/script/build_arrow.sh b/arrow-data-source/script/build_arrow.sh deleted file mode 100755 index 410e310709a2..000000000000 --- a/arrow-data-source/script/build_arrow.sh +++ /dev/null @@ -1,106 +0,0 @@ -#!/bin/bash - -set -eu - -NPROC=$(nproc) - -TESTS=OFF -BUILD_ARROW=OFF -STATIC_ARROW=OFF -ARROW_ROOT=/usr/local - -for arg in "$@" -do - case $arg in - -t=*|--tests=*) - TESTS=("${arg#*=}") - shift # Remove argument name from processing - ;; - -a=*|--build_arrow=*) - BUILD_ARROW=("${arg#*=}") - shift # Remove argument name from processing - ;; - -s=*|--static_arrow=*) - STATIC_ARROW=("${arg#*=}") - shift # Remove argument name from processing - ;; - -ar=*|--arrow_root=*) - ARROW_ROOT=("${arg#*=}") - shift # Remove argument name from processing - ;; - *) - OTHER_ARGUMENTS+=("$1") - shift # Remove generic argument from processing - ;; - esac -done - -echo "CMAKE Arguments:" -echo "TESTS=${TESTS}" -echo "BUILD_ARROW=${BUILD_ARROW}" -echo "STATIC_ARROW=${STATIC_ARROW}" -echo "ARROW_ROOT=${ARROW_ROOT}" - -CURRENT_DIR=$(cd "$(dirname "$BASH_SOURCE")"; pwd) -echo $CURRENT_DIR - -cd ${CURRENT_DIR} -if [ -d build ]; then - rm -r build -fi - -if [ $BUILD_ARROW == "ON" ]; then -echo "Building Arrow from Source ..." -mkdir build -cd build -ARROW_PREFIX="${CURRENT_DIR}/build" # Use build directory as ARROW_PREFIX -ARROW_SOURCE_DIR="${ARROW_PREFIX}/arrow_ep" -ARROW_INSTALL_DIR="${ARROW_PREFIX}/arrow_install" - -echo "ARROW_PREFIX=${ARROW_PREFIX}" -echo "ARROW_SOURCE_DIR=${ARROW_SOURCE_DIR}" -echo "ARROW_INSTALL_DIR=${ARROW_INSTALL_DIR}" -mkdir -p $ARROW_SOURCE_DIR -mkdir -p $ARROW_INSTALL_DIR -git clone https://github.com/oap-project/arrow.git --branch arrow-4.0.0-oap $ARROW_SOURCE_DIR -pushd $ARROW_SOURCE_DIR - -cmake ./cpp \ - -DARROW_BUILD_STATIC=OFF -DARROW_BUILD_SHARED=ON -DARROW_COMPUTE=ON \ - -DARROW_S3=ON \ - -DARROW_GANDIVA_JAVA=ON \ - -DARROW_GANDIVA=ON \ - -DARROW_PARQUET=ON \ - -DARROW_HDFS=ON \ - -DARROW_BOOST_USE_SHARED=OFF \ - -DARROW_JNI=ON \ - -DARROW_DATASET=ON \ - -DARROW_WITH_PROTOBUF=ON \ - -DARROW_WITH_SNAPPY=ON \ - -DARROW_WITH_LZ4=ON \ - -DARROW_WITH_ZSTD=OFF \ - -DARROW_WITH_BROTLI=OFF \ - -DARROW_WITH_ZLIB=OFF \ - -DARROW_WITH_FASTPFOR=ON \ - -DARROW_FILESYSTEM=ON \ - -DARROW_JSON=ON \ - -DARROW_CSV=ON \ - -DARROW_FLIGHT=OFF \ - -DARROW_JEMALLOC=ON \ - -DARROW_SIMD_LEVEL=AVX2 \ - -DARROW_RUNTIME_SIMD_LEVEL=MAX \ - -DARROW_DEPENDENCY_SOURCE=BUNDLED \ - -DCMAKE_INSTALL_PREFIX=${ARROW_INSTALL_DIR} \ - -DCMAKE_INSTALL_LIBDIR=lib - -make -j$NPROC -make install - -cd java -mvn clean install -P arrow-jni -am -Darrow.cpp.build.dir=${ARROW_INSTALL_DIR}/lib -DskipTests -Dcheckstyle.skip -popd -echo "Finish to build Arrow from Source !!!" -else -echo "Use ARROW_ROOT as Arrow Library Path" -echo "ARROW_ROOT=${ARROW_ROOT}" -fi diff --git a/arrow-data-source/standard/pom.xml b/arrow-data-source/standard/pom.xml deleted file mode 100644 index fa35d662c6f8..000000000000 --- a/arrow-data-source/standard/pom.xml +++ /dev/null @@ -1,105 +0,0 @@ - - - - spark-arrow-datasource - com.intel.oap - 1.2.0-snapshot - - 4.0.0 - - spark-arrow-datasource-standard - - - - - com.intel.oap - spark-arrow-datasource-common - ${project.version} - - - - - ${project.basedir}/src/main/scala - ${project.basedir}/src/test/scala - - - org.apache.maven.plugins - maven-compiler-plugin - 3.8.0 - - 1.8 - 1.8 - - - - compile - - compile - - - - - - org.codehaus.mojo - build-helper-maven-plugin - - - add-src-1 - generate-sources - - add-source - - - - ${project.basedir}/src/main/java - - - - - - - maven-assembly-plugin - 3.3.0 - - - jar-with-dependencies - - - - - make-assembly - package - - single - - - - - - org.apache.maven.plugins - maven-source-plugin - - - attach-sources - - jar - - - - - - - - - - org.scala-tools - maven-scala-plugin - - ${scala.version} - - - - - diff --git a/arrow-data-source/standard/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/arrow-data-source/standard/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister deleted file mode 100644 index dfdfbba20034..000000000000 --- a/arrow-data-source/standard/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ /dev/null @@ -1 +0,0 @@ -com.intel.oap.spark.sql.execution.datasources.v2.arrow.ArrowDataSourceV2 \ No newline at end of file diff --git a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/ArrowWriteExtension.scala b/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/ArrowWriteExtension.scala deleted file mode 100644 index 7f1d6e1530ca..000000000000 --- a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/ArrowWriteExtension.scala +++ /dev/null @@ -1,163 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.intel.oap.spark.sql - -import com.intel.oap.spark.sql.ArrowWriteExtension.ArrowWritePostRule -import com.intel.oap.spark.sql.ArrowWriteExtension.DummyRule -import com.intel.oap.spark.sql.ArrowWriteExtension.SimpleColumnarRule -import com.intel.oap.spark.sql.ArrowWriteExtension.SimpleStrategy -import com.intel.oap.spark.sql.execution.datasources.arrow.ArrowFileFormat -import com.intel.oap.sql.execution.RowToArrowColumnarExec -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{SparkSession, SparkSessionExtensions, Strategy} -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.plans.logical.OrderPreservingUnaryNode - -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.ArrayData -import org.apache.spark.sql.catalyst.util.MapData -import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec -import org.apache.spark.sql.execution.CodegenSupport -import org.apache.spark.sql.execution.ColumnarRule -import org.apache.spark.sql.execution.ColumnarToRowExec -import org.apache.spark.sql.execution.ColumnarToRowTransition -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.command.DataWritingCommandExec -import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand -import org.apache.spark.sql.types.DataType -import org.apache.spark.sql.types.Decimal -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.unsafe.types.CalendarInterval -import org.apache.spark.unsafe.types.UTF8String - -class ArrowWriteExtension extends (SparkSessionExtensions => Unit) { - def apply(e: SparkSessionExtensions): Unit = { - e.injectColumnar(session => SimpleColumnarRule(DummyRule, ArrowWritePostRule(session))) - e.injectPlannerStrategy(session => SimpleStrategy()) - } -} - -object ArrowWriteExtension { - private object DummyRule extends Rule[SparkPlan] { - def apply(p: SparkPlan): SparkPlan = p - } - - private case class SimpleColumnarRule(pre: Rule[SparkPlan], post: Rule[SparkPlan]) - extends ColumnarRule { - override def preColumnarTransitions: Rule[SparkPlan] = pre - override def postColumnarTransitions: Rule[SparkPlan] = post - } - - case class ArrowWritePostRule(session: SparkSession) extends Rule[SparkPlan] { - override def apply(plan: SparkPlan): SparkPlan = plan match { - case rc @ DataWritingCommandExec(cmd, ColumnarToRowExec(child)) => - cmd match { - case command: InsertIntoHadoopFsRelationCommand => - if (command.fileFormat - .isInstanceOf[ArrowFileFormat]) { - rc.withNewChildren(Array(ColumnarToFakeRowAdaptor(child))) - } else { - plan.withNewChildren(plan.children.map(apply)) - } - case _ => plan.withNewChildren(plan.children.map(apply)) - } - case rc @ DataWritingCommandExec(cmd, child) => - cmd match { - case command: InsertIntoHadoopFsRelationCommand => - if (command.fileFormat - .isInstanceOf[ArrowFileFormat]) { - child match { - case c: AdaptiveSparkPlanExec => - rc.withNewChildren( - Array( - AdaptiveSparkPlanExec( - ColumnarToFakeRowAdaptor(c.inputPlan), - c.context, - c.preprocessingRules, - c.isSubquery))) - case other => - rc.withNewChildren( - Array(ColumnarToFakeRowAdaptor(RowToArrowColumnarExec(child)))) - } - } else { - plan.withNewChildren(plan.children.map(apply)) - } - case _ => plan.withNewChildren(plan.children.map(apply)) - } - case plan: SparkPlan => plan.withNewChildren(plan.children.map(apply)) - } - } - - class FakeRow(val batch: ColumnarBatch) extends InternalRow { - override def numFields: Int = throw new UnsupportedOperationException() - override def setNullAt(i: Int): Unit = throw new UnsupportedOperationException() - override def update(i: Int, value: Any): Unit = throw new UnsupportedOperationException() - override def copy(): InternalRow = throw new UnsupportedOperationException() - override def isNullAt(ordinal: Int): Boolean = throw new UnsupportedOperationException() - override def getBoolean(ordinal: Int): Boolean = throw new UnsupportedOperationException() - override def getByte(ordinal: Int): Byte = throw new UnsupportedOperationException() - override def getShort(ordinal: Int): Short = throw new UnsupportedOperationException() - override def getInt(ordinal: Int): Int = throw new UnsupportedOperationException() - override def getLong(ordinal: Int): Long = throw new UnsupportedOperationException() - override def getFloat(ordinal: Int): Float = throw new UnsupportedOperationException() - override def getDouble(ordinal: Int): Double = throw new UnsupportedOperationException() - override def getDecimal(ordinal: Int, precision: Int, scale: Int): Decimal = - throw new UnsupportedOperationException() - override def getUTF8String(ordinal: Int): UTF8String = - throw new UnsupportedOperationException() - override def getBinary(ordinal: Int): Array[Byte] = throw new UnsupportedOperationException() - override def getInterval(ordinal: Int): CalendarInterval = - throw new UnsupportedOperationException() - override def getStruct(ordinal: Int, numFields: Int): InternalRow = - throw new UnsupportedOperationException() - override def getArray(ordinal: Int): ArrayData = throw new UnsupportedOperationException() - override def getMap(ordinal: Int): MapData = throw new UnsupportedOperationException() - override def get(ordinal: Int, dataType: DataType): AnyRef = - throw new UnsupportedOperationException() - } - - private case class ColumnarToFakeRowLogicAdaptor(child: LogicalPlan) - extends OrderPreservingUnaryNode { - override def output: Seq[Attribute] = child.output - } - - private case class ColumnarToFakeRowAdaptor(child: SparkPlan) extends ColumnarToRowTransition { - if (!child.logicalLink.isEmpty) { - setLogicalLink(ColumnarToFakeRowLogicAdaptor(child.logicalLink.get)) - } - - override protected def doExecute(): RDD[InternalRow] = { - child.executeColumnar().map { cb => new FakeRow(cb) } - } - - override def output: Seq[Attribute] = child.output - } - - case class SimpleStrategy() extends Strategy { - override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case ColumnarToFakeRowLogicAdaptor(child: LogicalPlan) => - Seq(ColumnarToFakeRowAdaptor(planLater(child))) - case other => - Nil - } - } - -} diff --git a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/ArrowWriteQueue.scala b/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/ArrowWriteQueue.scala deleted file mode 100644 index d03ab27b06ff..000000000000 --- a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/ArrowWriteQueue.scala +++ /dev/null @@ -1,134 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.intel.oap.spark.sql - -import java.lang -import java.net.URI -import java.util.Collections -import java.util.UUID -import java.util.concurrent.ArrayBlockingQueue -import java.util.concurrent.TimeUnit -import java.util.regex.Pattern - -import com.intel.oap.spark.sql.ArrowWriteQueue.EOS_BATCH -import com.intel.oap.spark.sql.ArrowWriteQueue.ScannerImpl -import org.apache.arrow.dataset.file.DatasetFileWriter -import org.apache.arrow.dataset.file.format.FileFormat -import org.apache.arrow.dataset.scanner.Scanner -import org.apache.arrow.dataset.scanner.ScanTask -import org.apache.arrow.vector.ipc.message.ArrowRecordBatch -import org.apache.arrow.vector.types.pojo.Schema - -class ArrowWriteQueue(schema: Schema, fileFormat: FileFormat, outputFileURI: String) - extends AutoCloseable { - private val scanner = new ScannerImpl(schema) - - private val writeThread = new Thread(() => { - URI.create(outputFileURI) // validate uri - val matcher = ArrowWriteQueue.TAILING_FILENAME_REGEX.matcher(outputFileURI) - if (!matcher.matches()) { - throw new IllegalArgumentException("illegal out put file uri: " + outputFileURI) - } - val dirURI = matcher.group(1) - val fileName = matcher.group(2) - - DatasetFileWriter.write(scanner, fileFormat, dirURI, Array(), 1, fileName) - }, "ArrowWriteQueue - " + UUID.randomUUID().toString) - - writeThread.start() - - def enqueue(batch: ArrowRecordBatch): Unit = { - scanner.enqueue(batch) - } - - override def close(): Unit = { - scanner.enqueue(EOS_BATCH) - writeThread.join() - } -} - -object ArrowWriteQueue { - private val TAILING_FILENAME_REGEX = Pattern.compile("^(.*)/([^/]+)$") - private val EOS_BATCH = new ArrowRecordBatch(0, Collections.emptyList(), Collections.emptyList()) - - class ScannerImpl(schema: Schema) extends Scanner { - private val writeQueue = new ArrayBlockingQueue[ArrowRecordBatch](64) - - def enqueue(batch: ArrowRecordBatch): Unit = { - writeQueue.put(batch) - } - - override def scan(): lang.Iterable[_ <: ScanTask] = { - Collections.singleton(new ScanTask { - override def execute(): ScanTask.BatchIterator = { - new ScanTask.BatchIterator { - private var currentBatch: Option[ArrowRecordBatch] = None - - override def hasNext: Boolean = { - if (currentBatch.isDefined) { - return true - } - val batch = try { - writeQueue.poll(30L, TimeUnit.MINUTES) - } catch { - case _: InterruptedException => - Thread.currentThread().interrupt() - EOS_BATCH - } - if (batch == null) { - throw new RuntimeException("ArrowWriter: Timeout waiting for data") - } - if (batch == EOS_BATCH) { - return false - } - currentBatch = Some(batch) - true - } - - override def next(): ArrowRecordBatch = { - if (currentBatch.isEmpty) { - throw new IllegalStateException() - } - try { - currentBatch.get - } finally { - currentBatch = None - } - } - - override def close(): Unit = { - - } - } - } - - override def close(): Unit = { - - } - }) - } - - override def schema(): Schema = { - schema - } - - override def close(): Unit = { - - } - } -} diff --git a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/DataFrameReaderImplicits.scala b/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/DataFrameReaderImplicits.scala deleted file mode 100644 index ee3b5f291fc7..000000000000 --- a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/DataFrameReaderImplicits.scala +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.intel.oap.spark.sql - -import org.apache.spark.sql.{DataFrame, DataFrameReader} - -class DataFrameReaderImplicits(r: DataFrameReader) { - - /** - * Loads a file via Arrow Datasets API and returns the result as a `DataFrame`. - * - * @param path input path - * @since 3.0.0-SNAPSHOT - */ - def arrow(path: String): DataFrame = { - // This method ensures that calls that explicit need single argument works, see SPARK-16009 - arrow(Seq(path): _*) - } - - /** - * Loads files via Arrow Datasets API and returns the result as a `DataFrame`. - * - * @param paths input paths - * @since 3.0.0-SNAPSHOT - */ - @scala.annotation.varargs - def arrow(paths: String*): DataFrame = r.format("arrow").load(paths: _*) -} - -object DataFrameReaderImplicits { - implicit def readerConverter(r: DataFrameReader): DataFrameReaderImplicits = { - new DataFrameReaderImplicits(r) - } -} diff --git a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/DataFrameWriterImplicits.scala b/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/DataFrameWriterImplicits.scala deleted file mode 100644 index 4df6135b9468..000000000000 --- a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/DataFrameWriterImplicits.scala +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.intel.oap.spark.sql - -import org.apache.spark.sql.DataFrameWriter - -class DataFrameWriterImplicits[T](w: DataFrameWriter[T]) { - - def arrow(path: String): Unit = { - // This method ensures that calls that explicit need single argument works, see SPARK-16009 - w.format("arrow").save(path) - } -} - -object DataFrameWriterImplicits { - implicit def writerConverter[T](w: DataFrameWriter[T]): DataFrameWriterImplicits[T] = { - new DataFrameWriterImplicits[T](w) - } -} diff --git a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/arrow/ArrowFileFormat.scala b/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/arrow/ArrowFileFormat.scala deleted file mode 100644 index 47a21048e575..000000000000 --- a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/arrow/ArrowFileFormat.scala +++ /dev/null @@ -1,171 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.intel.oap.spark.sql.execution.datasources.arrow - -import java.net.URLDecoder - -import scala.collection.JavaConverters._ - -import com.intel.oap.spark.sql.ArrowWriteExtension.FakeRow -import com.intel.oap.spark.sql.ArrowWriteQueue -import com.intel.oap.spark.sql.execution.datasources.v2.arrow.{ArrowFilters, ArrowOptions, ArrowUtils} -import com.intel.oap.spark.sql.execution.datasources.v2.arrow.ArrowSQLConf._ -import com.intel.oap.vectorized.ArrowWritableColumnVector -import org.apache.arrow.dataset.scanner.ScanOptions -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.hadoop.mapreduce.Job -import org.apache.hadoop.mapreduce.TaskAttemptContext -import org.apache.parquet.hadoop.codec.CodecConfig - -import org.apache.spark.TaskContext -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriterFactory, PartitionedFile} -import org.apache.spark.sql.execution.datasources.OutputWriter -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils.UnsafeItr -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkVectorUtils -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.{DataSourceRegister, Filter} -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.CaseInsensitiveStringMap -import org.apache.spark.sql.vectorized.ColumnarBatch; - -class ArrowFileFormat extends FileFormat with DataSourceRegister with Serializable { - - - override def isSplitable(sparkSession: SparkSession, - options: Map[String, String], path: Path): Boolean = { - ArrowUtils.isOriginalFormatSplitable( - new ArrowOptions(new CaseInsensitiveStringMap(options.asJava).asScala.toMap)) - } - - def convert(files: Seq[FileStatus], options: Map[String, String]): Option[StructType] = { - ArrowUtils.readSchema(files, new CaseInsensitiveStringMap(options.asJava)) - } - - override def inferSchema(sparkSession: SparkSession, - options: Map[String, String], - files: Seq[FileStatus]): Option[StructType] = { - convert(files, options) - } - - override def prepareWrite(sparkSession: SparkSession, - job: Job, - options: Map[String, String], - dataSchema: StructType): OutputWriterFactory = { - val arrowOptions = new ArrowOptions(new CaseInsensitiveStringMap(options.asJava).asScala.toMap) - new OutputWriterFactory { - override def getFileExtension(context: TaskAttemptContext): String = { - ArrowUtils.getFormat(arrowOptions) match { - case _: org.apache.arrow.dataset.file.format.ParquetFileFormat => - CodecConfig.from(context).getCodec.getExtension + ".parquet" - case f => throw new IllegalArgumentException("Unimplemented file type to write: " + f) - } - } - - override def newInstance(path: String, dataSchema: StructType, - context: TaskAttemptContext): OutputWriter = { - val writeQueue = new ArrowWriteQueue(ArrowUtils.toArrowSchema(dataSchema), - ArrowUtils.getFormat(arrowOptions), path) - - new OutputWriter { - override def write(row: InternalRow): Unit = { - val batch = row.asInstanceOf[FakeRow].batch - writeQueue.enqueue(SparkVectorUtils - .toArrowRecordBatch(batch)) - } - - override def close(): Unit = { - writeQueue.close() - } - } - } - } - } - - override def supportBatch(sparkSession: SparkSession, dataSchema: StructType): Boolean = true - - override def buildReaderWithPartitionValues(sparkSession: SparkSession, - dataSchema: StructType, - partitionSchema: StructType, - requiredSchema: StructType, - filters: Seq[Filter], - options: Map[String, String], - hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { - val sqlConf = sparkSession.sessionState.conf; - val batchSize = sqlConf.parquetVectorizedReaderBatchSize - val enableFilterPushDown = sqlConf.arrowFilterPushDown - - (file: PartitionedFile) => { - val factory = ArrowUtils.makeArrowDiscovery( - URLDecoder.decode(file.filePath, "UTF-8"), file.start, file.length, - new ArrowOptions( - new CaseInsensitiveStringMap( - options.asJava).asScala.toMap)) - - // todo predicate validation / pushdown - val dataset = factory.finish(); - - val filter = if (enableFilterPushDown) { - ArrowFilters.translateFilters(filters) - } else { - org.apache.arrow.dataset.filter.Filter.EMPTY - } - - val scanOptions = new ScanOptions(requiredSchema.map(f => f.name).toArray, - filter, batchSize) - val scanner = dataset.newScan(scanOptions) - - val taskList = scanner - .scan() - .iterator() - .asScala - .toList - val itrList = taskList - .map(task => task.execute()) - - Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => { - itrList.foreach(_.close()) - taskList.foreach(_.close()) - scanner.close() - dataset.close() - factory.close() - })) - - val itr = itrList - .toIterator - .flatMap(itr => itr.asScala) - .map(batch => ArrowUtils.loadBatch(batch, file.partitionValues, partitionSchema, - requiredSchema)) - new UnsafeItr(itr).asInstanceOf[Iterator[InternalRow]] - } - } - - override def vectorTypes(requiredSchema: StructType, partitionSchema: StructType, - sqlConf: SQLConf): Option[Seq[String]] = { - Option(Seq.fill(requiredSchema.fields.length + partitionSchema.fields.length)( - classOf[ArrowWritableColumnVector].getName - )) - } - - override def shortName(): String = "arrow" -} - -object ArrowFileFormat { -} diff --git a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowDataSourceV2.scala b/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowDataSourceV2.scala deleted file mode 100644 index b688c06d8258..000000000000 --- a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowDataSourceV2.scala +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.intel.oap.spark.sql.execution.datasources.v2.arrow - -import com.intel.oap.spark.sql.execution.datasources.arrow.ArrowFileFormat - -import org.apache.spark.sql.connector.catalog.Table -import org.apache.spark.sql.execution.datasources.FileFormat -import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 -import org.apache.spark.sql.util.CaseInsensitiveStringMap - -class ArrowDataSourceV2 extends FileDataSourceV2 { - - private val format = classOf[ArrowFileFormat] - - override def fallbackFileFormat: Class[_ <: FileFormat] = { - format - } - - override def getTable(options: CaseInsensitiveStringMap): Table = { - val paths = getPaths(options) - val tableName = getTableName(options, paths) - ArrowTable(tableName, sparkSession, options, paths, None, fallbackFileFormat) - } - - override def shortName(): String = "arrow" -} diff --git a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowFilters.scala b/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowFilters.scala deleted file mode 100644 index f33c7995a852..000000000000 --- a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowFilters.scala +++ /dev/null @@ -1,206 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.intel.oap.spark.sql.execution.datasources.v2.arrow - -import org.apache.arrow.dataset.DatasetTypes -import org.apache.arrow.dataset.DatasetTypes.TreeNode -import org.apache.arrow.dataset.filter.FilterImpl - -import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.StructType - -object ArrowFilters { - def pruneWithSchema(pushedFilters: Array[Filter], schema: StructType): Seq[Filter] = { - pushedFilters.filter(pushedFilter => { - isToBeAccepted(pushedFilter, schema) - }) - } - - private def isToBeAccepted(pushedFilter: Filter, schema: StructType): Boolean = { - pushedFilter match { - case EqualTo(attribute, value) => existsIn(attribute, schema) - case GreaterThan(attribute, value) => existsIn(attribute, schema) - case GreaterThanOrEqual(attribute, value) => existsIn(attribute, schema) - case LessThan(attribute, value) => existsIn(attribute, schema) - case LessThanOrEqual(attribute, value) => existsIn(attribute, schema) - case Not(child) => isToBeAccepted(child, schema) - case And(left, right) => isToBeAccepted(left, schema) && isToBeAccepted(right, schema) - case Or(left, right) => isToBeAccepted(left, schema) && isToBeAccepted(right, schema) - case IsNotNull(attribute) => existsIn(attribute, schema) - case IsNull(attribute) => existsIn(attribute, schema) - case _ => false // fixme complete this - } - } - - private def existsIn(attr: String, schema: StructType): Boolean = { - schema.foreach(f => { - if (f.name == attr) { - return true; - } - }) - false - } - - def translateFilters(pushedFilters: Seq[Filter]): org.apache.arrow.dataset.filter.Filter = { - val node = pushedFilters - .flatMap(translateFilter) - .reduceOption((t1: TreeNode, t2: TreeNode) => { - DatasetTypes.TreeNode.newBuilder.setAndNode( - DatasetTypes.AndNode.newBuilder() - .setLeftArg(t1) - .setRightArg(t2) - .build()).build() - }) - if (node.isDefined) { - new FilterImpl(DatasetTypes.Condition.newBuilder() - .setRoot(node.get).build) - } else { - org.apache.arrow.dataset.filter.Filter.EMPTY - } - } - - private def translateValue(value: Any): Option[TreeNode] = { - value match { - case v: Integer => Some( - DatasetTypes.TreeNode.newBuilder.setIntNode( - DatasetTypes.IntNode.newBuilder.setValue(v).build) - .build) - case v: Long => Some( - DatasetTypes.TreeNode.newBuilder.setLongNode( - DatasetTypes.LongNode.newBuilder.setValue(v).build) - .build) - case v: Float => Some( - DatasetTypes.TreeNode.newBuilder.setFloatNode( - DatasetTypes.FloatNode.newBuilder.setValue(v).build) - .build) - case v: Double => Some( - DatasetTypes.TreeNode.newBuilder.setDoubleNode( - DatasetTypes.DoubleNode.newBuilder.setValue(v).build) - .build) - case v: Boolean => Some( - DatasetTypes.TreeNode.newBuilder.setBooleanNode( - DatasetTypes.BooleanNode.newBuilder.setValue(v).build) - .build) - case _ => None // fixme complete this - } - } - - private def translateFilter(pushedFilter: Filter): Option[TreeNode] = { - pushedFilter match { - case EqualTo(attribute, value) => - createComparisonNode("equal", attribute, value) - case GreaterThan(attribute, value) => - createComparisonNode("greater", attribute, value) - case GreaterThanOrEqual(attribute, value) => - createComparisonNode("greater_equal", attribute, value) - case LessThan(attribute, value) => - createComparisonNode("less", attribute, value) - case LessThanOrEqual(attribute, value) => - createComparisonNode("less_equal", attribute, value) - case Not(child) => - createNotNode(child) - case And(left, right) => - createAndNode(left, right) - case Or(left, right) => - createOrNode(left, right) - case IsNotNull(attribute) => - createIsNotNullNode(attribute) - case IsNull(attribute) => - createIsNullNode(attribute) - case _ => None // fixme complete this - } - } - - private def createComparisonNode(opName: String, - attribute: String, value: Any): Option[TreeNode] = { - val translatedValue = translateValue(value) - translatedValue match { - case Some(v) => Some( - DatasetTypes.TreeNode.newBuilder.setCpNode( - DatasetTypes.ComparisonNode.newBuilder - .setOpName(opName) // todo make op names enumerable - .setLeftArg( - DatasetTypes.TreeNode.newBuilder.setFieldNode( - DatasetTypes.FieldNode.newBuilder.setName(attribute).build) - .build) - .setRightArg(v) - .build) - .build) - case None => None - } - } - - def createNotNode(child: Filter): Option[TreeNode] = { - val translatedChild = translateFilter(child) - if (translatedChild.isEmpty) { - return None - } - Some(DatasetTypes.TreeNode.newBuilder - .setNotNode(DatasetTypes.NotNode.newBuilder.setArgs(translatedChild.get).build()).build()) - } - - def createIsNotNullNode(attribute: String): Option[TreeNode] = { - Some(DatasetTypes.TreeNode.newBuilder - .setIsValidNode( - DatasetTypes.IsValidNode.newBuilder.setArgs( - DatasetTypes.TreeNode.newBuilder.setFieldNode( - DatasetTypes.FieldNode.newBuilder.setName(attribute).build) - .build).build()).build()) - } - - def createIsNullNode(attribute: String): Option[TreeNode] = { - Some(DatasetTypes.TreeNode.newBuilder - .setNotNode( - DatasetTypes.NotNode.newBuilder.setArgs( - DatasetTypes.TreeNode.newBuilder - .setIsValidNode( - DatasetTypes.IsValidNode.newBuilder.setArgs( - DatasetTypes.TreeNode.newBuilder.setFieldNode( - DatasetTypes.FieldNode.newBuilder.setName(attribute).build) - .build) - .build()).build()).build()).build()) - } - - def createAndNode(left: Filter, right: Filter): Option[TreeNode] = { - val translatedLeft = translateFilter(left) - val translatedRight = translateFilter(right) - if (translatedLeft.isEmpty || translatedRight.isEmpty) { - return None - } - Some(DatasetTypes.TreeNode.newBuilder - .setAndNode(DatasetTypes.AndNode.newBuilder - .setLeftArg(translatedLeft.get) - .setRightArg(translatedRight.get) - .build()) - .build()) - } - - def createOrNode(left: Filter, right: Filter): Option[TreeNode] = { - val translatedLeft = translateFilter(left) - val translatedRight = translateFilter(right) - if (translatedLeft.isEmpty || translatedRight.isEmpty) { - return None - } - Some(DatasetTypes.TreeNode.newBuilder - .setOrNode(DatasetTypes.OrNode.newBuilder - .setLeftArg(translatedLeft.get) - .setRightArg(translatedRight.get) - .build()) - .build()) - } -} diff --git a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowOptions.scala b/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowOptions.scala deleted file mode 100644 index efaf9a589ee9..000000000000 --- a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowOptions.scala +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.intel.oap.spark.sql.execution.datasources.v2.arrow - -import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap - -class ArrowOptions(val parameters: CaseInsensitiveMap[String]) - extends Serializable { - - def this(parameters: Map[String, String]) = this(CaseInsensitiveMap(parameters)) - - val originalFormat = parameters - .get(ArrowOptions.KEY_ORIGINAL_FORMAT) - .getOrElse(ArrowOptions.DEFAULT_ORIGINAL_FORMAT) - val targetFormat = parameters - .get(ArrowOptions.KEY_TARGET_FORMAT) - .getOrElse(ArrowOptions.DEFAULT_TARGET_FORMAT) - - @deprecated - val filesystem = parameters - .get(ArrowOptions.KEY_FILESYSTEM) - .getOrElse(ArrowOptions.DEFAULT_FILESYSTEM) -} - -object ArrowOptions { - val KEY_ORIGINAL_FORMAT = "originalFormat" - val DEFAULT_ORIGINAL_FORMAT = "parquet" - val KEY_TARGET_FORMAT = "targetFormat" - val DEFAULT_TARGET_FORMAT = "parquet" - - @deprecated - val KEY_FILESYSTEM = "filesystem" - val DEFAULT_FILESYSTEM = "hdfs" -} diff --git a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowPartitionReaderFactory.scala b/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowPartitionReaderFactory.scala deleted file mode 100644 index 99ccd781a595..000000000000 --- a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowPartitionReaderFactory.scala +++ /dev/null @@ -1,129 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.intel.oap.spark.sql.execution.datasources.v2.arrow - -import java.net.URLDecoder - -import scala.collection.JavaConverters._ - -import com.intel.oap.spark.sql.execution.datasources.v2.arrow.ArrowPartitionReaderFactory.ColumnarBatchRetainer -import com.intel.oap.spark.sql.execution.datasources.v2.arrow.ArrowSQLConf._ -import org.apache.arrow.dataset.scanner.ScanOptions - -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader} -import org.apache.spark.sql.execution.datasources.PartitionedFile -import org.apache.spark.sql.execution.datasources.v2.FilePartitionReaderFactory -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.SerializableConfiguration - -case class ArrowPartitionReaderFactory( - sqlConf: SQLConf, - broadcastedConf: Broadcast[SerializableConfiguration], - readDataSchema: StructType, - readPartitionSchema: StructType, - pushedFilters: Array[Filter], - options: ArrowOptions) - extends FilePartitionReaderFactory { - - private val batchSize = sqlConf.parquetVectorizedReaderBatchSize - private val enableFilterPushDown: Boolean = sqlConf.arrowFilterPushDown - - override def supportColumnarReads(partition: InputPartition): Boolean = true - - override def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] = { - // disable row based read - throw new UnsupportedOperationException - } - - override def buildColumnarReader( - partitionedFile: PartitionedFile): PartitionReader[ColumnarBatch] = { - val path = partitionedFile.filePath - val factory = ArrowUtils.makeArrowDiscovery(URLDecoder.decode(path, "UTF-8"), - partitionedFile.start, partitionedFile.length, options) - val dataset = factory.finish() - val filter = if (enableFilterPushDown) { - ArrowFilters.translateFilters(ArrowFilters.pruneWithSchema(pushedFilters, readDataSchema)) - } else { - org.apache.arrow.dataset.filter.Filter.EMPTY - } - val scanOptions = new ScanOptions(readDataSchema.map(f => f.name).toArray, - filter, batchSize) - val scanner = dataset.newScan(scanOptions) - - val taskList = scanner - .scan() - .iterator() - .asScala - .toList - - val vsrItrList = taskList - .map(task => task.execute()) - - val batchItr = vsrItrList - .toIterator - .flatMap(itr => itr.asScala) - .map(batch => ArrowUtils.loadBatch(batch, partitionedFile.partitionValues, - readPartitionSchema, readDataSchema)) - - new PartitionReader[ColumnarBatch] { - val holder = new ColumnarBatchRetainer() - - override def next(): Boolean = { - holder.release() - batchItr.hasNext - } - - override def get(): ColumnarBatch = { - val batch = batchItr.next() - holder.retain(batch) - batch - } - - override def close(): Unit = { - holder.release() - vsrItrList.foreach(itr => itr.close()) - taskList.foreach(task => task.close()) - scanner.close() - dataset.close() - factory.close() - } - } - } -} - -object ArrowPartitionReaderFactory { - private class ColumnarBatchRetainer { - private var retained: Option[ColumnarBatch] = None - - def retain(batch: ColumnarBatch): Unit = { - if (retained.isDefined) { - throw new IllegalStateException - } - retained = Some(batch) - } - - def release(): Unit = { - retained.foreach(b => b.close()) - retained = None - } - } -} diff --git a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowSQLConf.scala b/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowSQLConf.scala deleted file mode 100644 index 3b0cb69325dc..000000000000 --- a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowSQLConf.scala +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.intel.oap.spark.sql.execution.datasources.v2.arrow - -import org.apache.spark.sql.internal.SQLConf - -object ArrowSQLConf { - val ARROW_FILTER_PUSHDOWN_ENABLED = SQLConf.buildConf("spark.sql.arrow.filterPushdown") - .doc("Enables Arrow filter push-down optimization when set to true.") - .booleanConf - .createWithDefault(true) - - implicit def fromSQLConf(c: SQLConf): ArrowSQLConf = { - new ArrowSQLConf(c) - } -} - -class ArrowSQLConf(c: SQLConf) { - def arrowFilterPushDown: Boolean = c.getConf(ArrowSQLConf.ARROW_FILTER_PUSHDOWN_ENABLED) -} diff --git a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowScan.scala b/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowScan.scala deleted file mode 100644 index 20e069e01108..000000000000 --- a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowScan.scala +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.intel.oap.spark.sql.execution.datasources.v2.arrow - -import scala.collection.JavaConverters._ - -import org.apache.hadoop.fs.Path - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.connector.read.PartitionReaderFactory -import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex -import org.apache.spark.sql.execution.datasources.v2.FileScan -import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.CaseInsensitiveStringMap -import org.apache.spark.util.SerializableConfiguration - -case class ArrowScan( - sparkSession: SparkSession, - fileIndex: PartitioningAwareFileIndex, - readDataSchema: StructType, - readPartitionSchema: StructType, - pushedFilters: Array[Filter], - options: CaseInsensitiveStringMap, - partitionFilters: Seq[Expression] = Seq.empty, - dataFilters: Seq[Expression] = Seq.empty) - extends FileScan { - - override def isSplitable(path: Path): Boolean = { - ArrowUtils.isOriginalFormatSplitable( - new ArrowOptions(new CaseInsensitiveStringMap(options).asScala.toMap)) - } - - override def createReaderFactory(): PartitionReaderFactory = { - val caseSensitiveMap = options.asCaseSensitiveMap().asScala.toMap - val hconf = sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap) - val broadcastedConf = - sparkSession.sparkContext.broadcast(new SerializableConfiguration(hconf)) - ArrowPartitionReaderFactory( - sparkSession.sessionState.conf, - broadcastedConf, - readDataSchema, - readPartitionSchema, - pushedFilters, - new ArrowOptions(options.asScala.toMap)) - } - - override def withFilters(partitionFilters: Seq[Expression], - dataFilters: Seq[Expression]): FileScan = - this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) -} diff --git a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowScanBuilder.scala b/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowScanBuilder.scala deleted file mode 100644 index 6583455e030f..000000000000 --- a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowScanBuilder.scala +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.intel.oap.spark.sql.execution.datasources.v2.arrow - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} -import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex -import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder -import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.CaseInsensitiveStringMap - -case class ArrowScanBuilder( - sparkSession: SparkSession, - fileIndex: PartitioningAwareFileIndex, - schema: StructType, - dataSchema: StructType, - options: CaseInsensitiveStringMap) - extends FileScanBuilder(sparkSession, fileIndex, dataSchema) - with SupportsPushDownFilters { - - private var filters: Array[Filter] = Array.empty - private lazy val pushedArrowFilters: Array[Filter] = { - filters // todo filter validation & pushdown - } - - override def pushFilters(filters: Array[Filter]): Array[Filter] = { - this.filters = filters - this.filters - } - - override def pushedFilters: Array[Filter] = pushedArrowFilters - - override def build(): Scan = { - ArrowScan( - sparkSession, - fileIndex, - readDataSchema(), - readPartitionSchema(), - pushedFilters, - options) - } -} diff --git a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowTable.scala b/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowTable.scala deleted file mode 100644 index c25864a538c0..000000000000 --- a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowTable.scala +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.intel.oap.spark.sql.execution.datasources.v2.arrow - -import org.apache.arrow.memory.AllocationListener -import org.apache.hadoop.fs.FileStatus - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connector.read.ScanBuilder -import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} -import org.apache.spark.sql.execution.datasources.FileFormat -import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.CaseInsensitiveStringMap - -case class ArrowTable( - name: String, - sparkSession: SparkSession, - options: CaseInsensitiveStringMap, - paths: Seq[String], - userSpecifiedSchema: Option[StructType], - fallbackFileFormat: Class[_ <: FileFormat]) - extends FileTable(sparkSession, options, paths, userSpecifiedSchema) { - - override def inferSchema(files: Seq[FileStatus]): Option[StructType] = { - ArrowUtils.readSchema(files, options) - } - - override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { - ArrowScanBuilder(sparkSession, fileIndex, schema, dataSchema, options) - } - - override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { - throw new UnsupportedOperationException // fixme implement later - } - - override def formatName: String = "ARROW" -} diff --git a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowUtils.scala b/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowUtils.scala deleted file mode 100644 index 07c572cddaf1..000000000000 --- a/arrow-data-source/standard/src/main/scala/com/intel/oap/spark/sql/execution/datasources/v2/arrow/ArrowUtils.scala +++ /dev/null @@ -1,142 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.intel.oap.spark.sql.execution.datasources.v2.arrow - -import java.net.URI -import java.time.ZoneId - -import scala.collection.JavaConverters._ - -import com.intel.oap.vectorized.ArrowWritableColumnVector -import org.apache.arrow.dataset.file.FileSystemDatasetFactory -import org.apache.arrow.vector.ipc.message.ArrowRecordBatch -import org.apache.arrow.vector.types.pojo.Schema -import org.apache.hadoop.fs.FileStatus - -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.datasources.v2.arrow.{SparkMemoryUtils, SparkSchemaUtils} -import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.CaseInsensitiveStringMap -import org.apache.spark.sql.vectorized.ColumnVector -import org.apache.spark.sql.vectorized.ColumnarBatch - -object ArrowUtils { - - def readSchema(file: FileStatus, options: CaseInsensitiveStringMap): Option[StructType] = { - val factory: FileSystemDatasetFactory = - makeArrowDiscovery(file.getPath.toString, -1L, -1L, - new ArrowOptions(options.asScala.toMap)) - val schema = factory.inspect() - try { - Option(SparkSchemaUtils.fromArrowSchema(schema)) - } finally { - factory.close() - } - } - - def readSchema(files: Seq[FileStatus], options: CaseInsensitiveStringMap): Option[StructType] = { - if (files.isEmpty) { - throw new IllegalArgumentException("No input file specified") - } - readSchema(files.toList.head, options) // todo merge schema - } - - def isOriginalFormatSplitable(options: ArrowOptions): Boolean = { - val format = getFormat(options) - format match { - case _: org.apache.arrow.dataset.file.format.ParquetFileFormat => - true - case _ => - false - } - } - - def makeArrowDiscovery(file: String, startOffset: Long, length: Long, - options: ArrowOptions): FileSystemDatasetFactory = { - - val format = getFormat(options) - val allocator = SparkMemoryUtils.contextAllocator() - val factory = new FileSystemDatasetFactory(allocator, - SparkMemoryUtils.contextMemoryPool(), - format, - rewriteUri(file), - startOffset, - length) - factory - } - - def toArrowSchema(t: StructType): Schema = { - // fixme this might be platform dependent - SparkSchemaUtils.toArrowSchema(t, SparkSchemaUtils.getLocalTimezoneID()) - } - - def loadBatch(input: ArrowRecordBatch, partitionValues: InternalRow, - partitionSchema: StructType, dataSchema: StructType): ColumnarBatch = { - val rowCount: Int = input.getLength - - val vectors = try { - ArrowWritableColumnVector.loadColumns(rowCount, toArrowSchema(dataSchema), input) - } finally { - input.close() - } - val partitionColumns = ArrowWritableColumnVector.allocateColumns(rowCount, partitionSchema) - (0 until partitionColumns.length).foreach(i => { - ColumnVectorUtils.populate(partitionColumns(i), partitionValues, i) - partitionColumns(i).setValueCount(rowCount) - partitionColumns(i).setIsConstant() - }) - - val batch = new ColumnarBatch( - vectors.map(_.asInstanceOf[ColumnVector]) ++ - partitionColumns.map(_.asInstanceOf[ColumnVector]), - rowCount) - batch - } - - def getFormat( - options: ArrowOptions): org.apache.arrow.dataset.file.format.FileFormat = { - val paramMap = options.parameters.toMap.asJava - options.originalFormat match { - case "parquet" => org.apache.arrow.dataset.file.format.ParquetFileFormat.create(paramMap) - case "csv" => org.apache.arrow.dataset.file.format.CsvFileFormat.create(paramMap) - case _ => throw new IllegalArgumentException("Unrecognizable format") - } - } - - private def rewriteUri(uriStr: String): String = { - val uri = URI.create(uriStr) - if (uri.getScheme == "s3" || uri.getScheme == "s3a") { - val s3Rewritten = new URI("s3", uri.getAuthority, - uri.getPath, uri.getQuery, uri.getFragment).toString - return s3Rewritten - } - val sch = uri.getScheme match { - case "hdfs" => "hdfs" - case "file" => "file" - } - val ssp = uri.getScheme match { - case "hdfs" => uri.getRawSchemeSpecificPart - case "file" => "//" + uri.getRawSchemeSpecificPart - } - val rewritten = new URI(sch, ssp, uri.getFragment) - rewritten.toString - } -} diff --git a/arrow-data-source/standard/src/test/resources/cars.csv b/arrow-data-source/standard/src/test/resources/cars.csv deleted file mode 100644 index 40ded573ade5..000000000000 --- a/arrow-data-source/standard/src/test/resources/cars.csv +++ /dev/null @@ -1,7 +0,0 @@ - -year,make,model,comment,blank -"2012","Tesla","S","No comment", - -1997,Ford,E350,"Go get one now they are going fast", -2015,Chevy,Volt - diff --git a/arrow-data-source/standard/src/test/resources/example-tab.csv b/arrow-data-source/standard/src/test/resources/example-tab.csv deleted file mode 100644 index f86f114fb45d..000000000000 --- a/arrow-data-source/standard/src/test/resources/example-tab.csv +++ /dev/null @@ -1,35 +0,0 @@ -id1 id2 id3 id4 id5 id6 v1 v2 v3 -id016 id016 id0000042202 15 24 5971 5 11 37.211254 -id039 id045 id0000029558 40 49 39457 5 4 48.951141 -id047 id023 id0000071286 68 20 74463 2 14 60.469241 -id043 id057 id0000015141 32 43 63743 1 15 7.692145 -id054 id040 id0000011083 9 25 16920 2 9 22.863525 -id029 id020 id0000017974 40 43 14435 3 13 87.521355 -id047 id023 id0000084849 90 96 35790 2 9 93.348148 -id091 id022 id0000031441 50 44 71525 3 11 81.013682 -id090 id048 id0000067778 24 2 51862 4 9 30.718821 -id070 id008 id0000091167 78 4 23333 5 15 70.95464 -id039 id084 id0000013708 94 81 44406 1 3 54.368009 -id023 id061 id0000011331 36 67 86498 5 2 13.847979 -id070 id054 id0000099110 24 15 47054 4 2 92.057305 -id022 id008 id0000038862 38 92 63088 3 10 33.517765 -id020 id070 id0000028952 17 57 50831 4 15 48.060814 -id078 id022 id0000082008 69 44 15891 1 4 95.75571 -id024 id033 id0000074157 1 57 83341 2 1 72.118722 -id053 id076 id0000061759 55 43 59469 5 10 10.574836 -id058 id087 id0000094028 14 49 72962 4 4 37.914258 -id095 id091 id0000066931 35 20 98979 3 3 16.733062 -id054 id061 id0000004843 69 58 14096 4 13 53.746802 -id019 id078 id0000047661 5 33 13347 5 5 95.013936 -id086 id088 id0000039469 45 86 65332 3 11 65.71087 -id021 id055 id0000035603 96 97 36475 4 9 90.835613 -id004 id034 id0000008260 99 8 73046 3 11 69.540405 -id053 id052 id0000008764 47 13 49231 1 15 32.039599 -id014 id050 id0000066034 45 32 33268 2 3 93.752279 -id099 id057 id0000062408 27 7 63984 5 6 77.454794 -id013 id067 id0000046109 69 90 21214 4 6 83.899656 -id042 id043 id0000025883 64 21 85711 4 14 84.141247 -id024 id062 id0000026824 79 16 49757 2 10 15.822967 -id058 id077 id0000016555 71 8 24728 3 9 92.085521 -id053 id012 id0000005595 73 28 79781 2 10 6.053862 -id100 id096 id0000073858 11 9 25962 1 10 87.268781 diff --git a/arrow-data-source/standard/src/test/resources/example.csv b/arrow-data-source/standard/src/test/resources/example.csv deleted file mode 100644 index 670d7c25ce57..000000000000 --- a/arrow-data-source/standard/src/test/resources/example.csv +++ /dev/null @@ -1,35 +0,0 @@ -id1,id2,id3,id4,id5,id6,v1,v2,v3 -id016,id016,id0000042202,15,24,5971,5,11,37.211254 -id039,id045,id0000029558,40,49,39457,5,4,48.951141 -id047,id023,id0000071286,68,20,74463,2,14,60.469241 -id043,id057,id0000015141,32,43,63743,1,15,7.692145 -id054,id040,id0000011083,9,25,16920,2,9,22.863525 -id029,id020,id0000017974,40,43,14435,3,13,87.521355 -id047,id023,id0000084849,90,96,35790,2,9,93.348148 -id091,id022,id0000031441,50,44,71525,3,11,81.013682 -id090,id048,id0000067778,24,2,51862,4,9,30.718821 -id070,id008,id0000091167,78,4,23333,5,15,70.95464 -id039,id084,id0000013708,94,81,44406,1,3,54.368009 -id023,id061,id0000011331,36,67,86498,5,2,13.847979 -id070,id054,id0000099110,24,15,47054,4,2,92.057305 -id022,id008,id0000038862,38,92,63088,3,10,33.517765 -id020,id070,id0000028952,17,57,50831,4,15,48.060814 -id078,id022,id0000082008,69,44,15891,1,4,95.75571 -id024,id033,id0000074157,1,57,83341,2,1,72.118722 -id053,id076,id0000061759,55,43,59469,5,10,10.574836 -id058,id087,id0000094028,14,49,72962,4,4,37.914258 -id095,id091,id0000066931,35,20,98979,3,3,16.733062 -id054,id061,id0000004843,69,58,14096,4,13,53.746802 -id019,id078,id0000047661,5,33,13347,5,5,95.013936 -id086,id088,id0000039469,45,86,65332,3,11,65.71087 -id021,id055,id0000035603,96,97,36475,4,9,90.835613 -id004,id034,id0000008260,99,8,73046,3,11,69.540405 -id053,id052,id0000008764,47,13,49231,1,15,32.039599 -id014,id050,id0000066034,45,32,33268,2,3,93.752279 -id099,id057,id0000062408,27,7,63984,5,6,77.454794 -id013,id067,id0000046109,69,90,21214,4,6,83.899656 -id042,id043,id0000025883,64,21,85711,4,14,84.141247 -id024,id062,id0000026824,79,16,49757,2,10,15.822967 -id058,id077,id0000016555,71,8,24728,3,9,92.085521 -id053,id012,id0000005595,73,28,79781,2,10,6.053862 -id100,id096,id0000073858,11,9,25962,1,10,87.268781 diff --git a/arrow-data-source/standard/src/test/resources/people.csv b/arrow-data-source/standard/src/test/resources/people.csv deleted file mode 100644 index 4d9b27bf9ac8..000000000000 --- a/arrow-data-source/standard/src/test/resources/people.csv +++ /dev/null @@ -1,3 +0,0 @@ -name,age,job -Jorge,30,Developer -Bob,32,Developer diff --git a/arrow-data-source/standard/src/test/scala/com/intel/oap/spark/sql/execution/datasources/arrow/ArrowDataSourceTPCHBasedTest.scala b/arrow-data-source/standard/src/test/scala/com/intel/oap/spark/sql/execution/datasources/arrow/ArrowDataSourceTPCHBasedTest.scala deleted file mode 100644 index b33d4a1d32cb..000000000000 --- a/arrow-data-source/standard/src/test/scala/com/intel/oap/spark/sql/execution/datasources/arrow/ArrowDataSourceTPCHBasedTest.scala +++ /dev/null @@ -1,286 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.intel.oap.spark.sql.execution.datasources.arrow - -import java.util.concurrent.{Executors, TimeUnit} - -import com.intel.oap.spark.sql.DataFrameReaderImplicits._ -import com.intel.oap.spark.sql.execution.datasources.v2.arrow.{ArrowOptions, ArrowUtils} - -import org.apache.spark.SparkConf -import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSparkSession - -class ArrowDataSourceTPCHBasedTest extends QueryTest with SharedSparkSession { - - // tpc-h query cases: generated tpc-h dataset required - private val prefix = "/root/Downloads/" - private val tpchFolder = "date_tpch_10" - private val lineitem = prefix + tpchFolder + "/lineitem" - private val part = prefix + tpchFolder + "/part" - private val partSupp = prefix + tpchFolder + "/partsupp" - private val supplier = prefix + tpchFolder + "/supplier" - private val orders = prefix + tpchFolder + "/orders" - private val nation = prefix + tpchFolder + "/nation" - - - override protected def sparkConf: SparkConf = { - val conf = super.sparkConf - conf.set("spark.memory.offHeap.size", String.valueOf(128 * 1024 * 1024)) - conf - } - - ignore("tpch lineitem - desc") { - val frame = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .option(ArrowOptions.KEY_FILESYSTEM, "hdfs") - .arrow(lineitem) - frame.createOrReplaceTempView("lineitem") - - spark.sql("describe lineitem").show() - } - - ignore("tpch part - special characters in path") { - val frame = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .option(ArrowOptions.KEY_FILESYSTEM, "hdfs") - .arrow(part) - frame.createOrReplaceTempView("part") - - spark.sql("select * from part limit 100").show() - } - - ignore("tpch lineitem - read partition values") { - val frame = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .option(ArrowOptions.KEY_FILESYSTEM, "hdfs") - .arrow(orders) - frame.createOrReplaceTempView("orders") - - spark.sql("select o_orderdate from orders limit 100").show() - } - - ignore("tpch lineitem - asterisk select") { - val frame = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .option(ArrowOptions.KEY_FILESYSTEM, "hdfs") - .arrow(lineitem) - frame.createOrReplaceTempView("lineitem") - - spark.sql("select * from lineitem limit 10").show() - } - - ignore("tpch query 6") { - val frame = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .option(ArrowOptions.KEY_FILESYSTEM, "hdfs") - .arrow(lineitem) - frame.createOrReplaceTempView("lineitem") - - spark.sql("select\n\tsum(l_extendedprice * l_discount) as revenue\n" + - "from\n\tlineitem\n" + - "where\n\tl_shipdate >= date '1994-01-01'\n\t" + - "and l_shipdate < date '1994-01-01' + interval '1' year\n\t" + - "and l_discount between .06 - 0.01 and .06 + 0.01\n\t" + - "and l_quantity < 24").show() - } - - ignore("tpch query 6 - performance comparision") { - val iterations = 10 - withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "false") { - val frame1 = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .option(ArrowOptions.KEY_FILESYSTEM, "hdfs") - .arrow(lineitem) - frame1.createOrReplaceTempView("lineitem_arrow") - - val frame2 = spark.read - .parquet(lineitem) - frame2.createOrReplaceTempView("lineitem_parquet") - - val pPrev = System.currentTimeMillis() - (0 until iterations).foreach(_ => - spark.sql("select\n\tsum(l_extendedprice * l_discount) as revenue\n" + - "from\n\tlineitem_parquet\n" + - "where\n\tl_shipdate >= date '1994-01-01'\n\t" + - "and l_shipdate < date '1994-01-01' + interval '1' year\n\t" + - "and l_discount between .06 - 0.01 and .06 + 0.01\n\t" + - "and l_quantity < 24").show() - ) - val parquetExecTime = System.currentTimeMillis() - pPrev - - val aPrev = System.currentTimeMillis() - (0 until iterations).foreach(_ => { - // scalastyle:off println - println(SparkMemoryUtils.contextAllocator().getAllocatedMemory) - // scalastyle:on println - spark.sql("select\n\tsum(l_extendedprice * l_discount) as revenue\n" + - "from\n\tlineitem_arrow\n" + - "where\n\tl_shipdate >= date '1994-01-01'\n\t" + - "and l_shipdate < date '1994-01-01' + interval '1' year\n\t" + - "and l_discount between .06 - 0.01 and .06 + 0.01\n\t" + - "and l_quantity < 24").show() - } - ) - val arrowExecTime = System.currentTimeMillis() - aPrev - - // unstable assert - assert(arrowExecTime < parquetExecTime) - } - } - - ignore("tpch query 16 - performance comparision") { - val iterations = 1 - withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "false") { - val frame1 = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .option(ArrowOptions.KEY_FILESYSTEM, "hdfs") - .arrow(partSupp) - frame1.createOrReplaceTempView("partsupp_arrow") - - val frame2 = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .option(ArrowOptions.KEY_FILESYSTEM, "hdfs") - .arrow(part) - frame2.createOrReplaceTempView("part_arrow") - - val frame3 = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .option(ArrowOptions.KEY_FILESYSTEM, "hdfs") - .arrow(supplier) - frame3.createOrReplaceTempView("supplier_arrow") - - val frame4 = spark.read - .parquet(partSupp) - frame4.createOrReplaceTempView("partsupp_parquet") - - val frame5 = spark.read - .parquet(part) - frame5.createOrReplaceTempView("part_parquet") - - val frame6 = spark.read - .parquet(supplier) - frame6.createOrReplaceTempView("supplier_parquet") - - val pPrev = System.currentTimeMillis() - (0 until iterations).foreach(_ => - spark.sql("select\n\tp_brand,\n\tp_type,\n\tp_size," + - "\n\tcount(distinct ps_suppkey) as supplier_cnt\n" + - "from\n\tpartsupp_parquet,\n\tpart_parquet\nwhere\n\tp_partkey" + - " = ps_partkey\n\tand p_brand <> 'Brand#45'\n\t" + - "and p_type not like 'MEDIUM POLISHED%'\n\tand p_size in " + - "(49, 14, 23, 45, 19, 3, 36, 9)\n\t" + - "and ps_suppkey not in (\n\t\tselect\n\t\t\ts_suppkey\n\t\t" + - "from\n\t\t\tsupplier_parquet\n\t\twhere\n\t\t\t" + - "s_comment like '%Customer%Complaints%'\n\t)\ngroup by\n\t" + - "p_brand,\n\tp_type,\n\tp_size\norder by\n\t" + - "supplier_cnt desc,\n\tp_brand,\n\tp_type,\n\tp_size").show() - ) - val parquetExecTime = System.currentTimeMillis() - pPrev - - val aPrev = System.currentTimeMillis() - (0 until iterations).foreach(_ => - spark.sql("select\n\tp_brand,\n\tp_type,\n\tp_size," + - "\n\tcount(distinct ps_suppkey) as supplier_cnt\n" + - "from\n\tpartsupp_arrow,\n\tpart_arrow\nwhere\n\tp_partkey" + - " = ps_partkey\n\tand p_brand <> 'Brand#45'\n\t" + - "and p_type not like 'MEDIUM POLISHED%'\n\tand p_size in " + - "(49, 14, 23, 45, 19, 3, 36, 9)\n\t" + - "and ps_suppkey not in (\n\t\tselect\n\t\t\ts_suppkey\n\t\t" + - "from\n\t\t\tsupplier_arrow\n\t\twhere\n\t\t\t" + - "s_comment like '%Customer%Complaints%'\n\t)\ngroup by\n\t" + - "p_brand,\n\tp_type,\n\tp_size\norder by\n\t" + - "supplier_cnt desc,\n\tp_brand,\n\tp_type,\n\tp_size").show() - ) - val arrowExecTime = System.currentTimeMillis() - aPrev - - // scalastyle:off println - println(arrowExecTime) - println(parquetExecTime) - // scalastyle:on println - // unstable assert - assert(arrowExecTime < parquetExecTime) - } - } - - ignore("tpch query 1") { - val frame = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .option(ArrowOptions.KEY_FILESYSTEM, "hdfs") - .arrow(lineitem) - frame.createOrReplaceTempView("lineitem") - - spark.sql("select\n\tl_returnflag,\n\tl_linestatus," + - "\n\tsum(l_quantity) as sum_qty,\n\t" + - "sum(l_extendedprice) as sum_base_price," + - "\n\tsum(l_extendedprice * (1 - l_discount)) as sum_disc_price,\n\t" + - "sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge," + - "\n\tavg(l_quantity) as avg_qty,\n\t" + - "avg(l_extendedprice) as avg_price,\n\tavg(l_discount) as avg_disc," + - "\n\tcount(*) as count_order\nfrom\n\t" + - "lineitem\nwhere\n\tl_shipdate <= date '1998-12-01' - interval '90' day" + - "\ngroup by\n\tl_returnflag,\n\t" + - "l_linestatus\norder by\n\tl_returnflag,\n\tl_linestatus").explain(true) - } - - ignore("tpch query 21 - memory leak") { - val frame1 = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .option(ArrowOptions.KEY_FILESYSTEM, "hdfs") - .arrow(supplier) - frame1.createOrReplaceTempView("supplier") - val frame2 = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .option(ArrowOptions.KEY_FILESYSTEM, "hdfs") - .arrow(lineitem) - frame2.createOrReplaceTempView("lineitem") - val frame3 = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .option(ArrowOptions.KEY_FILESYSTEM, "hdfs") - .arrow(orders) - frame3.createOrReplaceTempView("orders") - val frame4 = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .option(ArrowOptions.KEY_FILESYSTEM, "hdfs") - .arrow(nation) - frame4.createOrReplaceTempView("nation") - - Executors.newSingleThreadExecutor().execute(() => { - spark.sql("select\n\ts_name,\n\tcount(*) as numwait\nfrom\n\tsupplier,\n\t" + - "lineitem l1,\n\torders,\n\tnation\nwhere\n\ts_suppkey = l1.l_suppkey\n\t" + - "and o_orderkey = l1.l_orderkey\n\tand o_orderstatus = 'F'\n\tand " + - "l1.l_receiptdate > l1.l_commitdate\n\tand exists (\n\t\tselect\n\t\t\t*\n\t\tfrom\n\t\t\t" + - "lineitem l2\n\t\twhere\n\t\t\tl2.l_orderkey = l1.l_orderkey\n\t\t\tand " + - "l2.l_suppkey <> l1.l_suppkey\n\t)\n\tand not exists (\n\t\tselect\n\t\t\t*\n\t\t" + - "from\n\t\t\tlineitem l3\n\t\twhere\n\t\t\tl3.l_orderkey = l1.l_orderkey\n\t\t\t" + - "and l3.l_suppkey <> l1.l_suppkey\n\t\t\tand l3.l_receiptdate > " + - "l3.l_commitdate\n\t)\n\tand s_nationkey = n_nationkey\n\tand n_name = 'SAUDI ARABIA'\n" + - "group by\n\ts_name\norder by\n\tnumwait desc,\n\t" + - "s_name\nlimit 100").show() - }) - Executors.newSingleThreadScheduledExecutor().scheduleWithFixedDelay(() => { - println("[org.apache.spark.sql.util.ArrowUtils.rootAllocator] " + - "Allocated memory amount: " + SparkMemoryUtils.contextAllocator()) - println("[com.intel.oap.vectorized.ArrowWritableColumnVector.allocator] " + - "Allocated memory amount: " + SparkMemoryUtils.contextAllocator().getAllocatedMemory) - }, 0L, 100L, TimeUnit.MILLISECONDS) - Thread.sleep(60 * 60 * 1000L) - } - -} diff --git a/arrow-data-source/standard/src/test/scala/com/intel/oap/spark/sql/execution/datasources/arrow/ArrowDataSourceTest.scala b/arrow-data-source/standard/src/test/scala/com/intel/oap/spark/sql/execution/datasources/arrow/ArrowDataSourceTest.scala deleted file mode 100644 index 5ad7596b901d..000000000000 --- a/arrow-data-source/standard/src/test/scala/com/intel/oap/spark/sql/execution/datasources/arrow/ArrowDataSourceTest.scala +++ /dev/null @@ -1,373 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.intel.oap.spark.sql.execution.datasources.arrow - -import java.io.File -import java.lang.management.ManagementFactory - -import com.intel.oap.spark.sql.ArrowWriteExtension -import com.intel.oap.spark.sql.DataFrameReaderImplicits._ -import com.intel.oap.spark.sql.DataFrameWriterImplicits._ -import com.intel.oap.spark.sql.execution.datasources.v2.arrow.ArrowOptions -import com.sun.management.UnixOperatingSystemMXBean -import org.apache.commons.io.FileUtils - -import org.apache.spark.SparkConf -import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.{DataFrame, QueryTest, Row} -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils -import org.apache.spark.sql.functions.col -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.StaticSQLConf.SPARK_SESSION_EXTENSIONS -import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType} - -class ArrowDataSourceTest extends QueryTest with SharedSparkSession { - private val parquetFile1 = "parquet-1.parquet" - private val parquetFile2 = "parquet-2.parquet" - private val parquetFile3 = "parquet-3.parquet" - private val parquetFile4 = "parquet-4.parquet" - private val parquetFile5 = "parquet-5.parquet" - - override protected def sparkConf: SparkConf = { - val conf = super.sparkConf - conf.set("spark.memory.offHeap.size", String.valueOf(10 * 1024 * 1024)) - conf.set("spark.unsafe.exceptionOnMemoryLeak", "false") - conf.set(SPARK_SESSION_EXTENSIONS.key, classOf[ArrowWriteExtension].getCanonicalName) - conf - } - - override def beforeAll(): Unit = { - super.beforeAll() - import testImplicits._ - spark.read - .json(Seq("{\"col\": -1}", "{\"col\": 0}", "{\"col\": 1}", "{\"col\": 2}", "{\"col\": null}") - .toDS()) - .repartition(1) - .write - .mode("overwrite") - .parquet(ArrowDataSourceTest.locateResourcePath(parquetFile1)) - - spark.read - .json(Seq("{\"col\": \"a\"}", "{\"col\": \"b\"}") - .toDS()) - .repartition(1) - .write - .mode("overwrite") - .parquet(ArrowDataSourceTest.locateResourcePath(parquetFile2)) - - spark.read - .json(Seq("{\"col1\": \"apple\", \"col2\": 100}", "{\"col1\": \"pear\", \"col2\": 200}", - "{\"col1\": \"apple\", \"col2\": 300}") - .toDS()) - .repartition(1) - .write - .mode("overwrite") - .parquet(ArrowDataSourceTest.locateResourcePath(parquetFile3)) - - spark.range(1000) - .select(col("id"), col("id").as("k")) - .write - .partitionBy("k") - .format("parquet") - .mode("overwrite") - .parquet(ArrowDataSourceTest.locateResourcePath(parquetFile4)) - - spark.range(100) - .select(col("id"), col("id").as("k")) - .write - .partitionBy("k") - .format("parquet") - .mode("overwrite") - .parquet(ArrowDataSourceTest.locateResourcePath(parquetFile5)) - - } - - override def afterAll(): Unit = { - delete(ArrowDataSourceTest.locateResourcePath(parquetFile1)) - delete(ArrowDataSourceTest.locateResourcePath(parquetFile2)) - delete(ArrowDataSourceTest.locateResourcePath(parquetFile3)) - delete(ArrowDataSourceTest.locateResourcePath(parquetFile4)) - delete(ArrowDataSourceTest.locateResourcePath(parquetFile5)) - super.afterAll() - } - - test("read parquet file") { - val path = ArrowDataSourceTest.locateResourcePath(parquetFile1) - verifyFrame( - spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .arrow(path), 5, 1) - } - - ignore("simple sql query on s3") { - val path = "s3a://mlp-spark-dataset-bucket/test_arrowds_s3_small" - val frame = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .arrow(path) - frame.createOrReplaceTempView("stab") - assert(spark.sql("select id from stab").count() === 1000) - } - - test("create catalog table") { - val path = ArrowDataSourceTest.locateResourcePath(parquetFile1) - spark.catalog.createTable("ptab", path, "arrow") - val sql = "select * from ptab" - spark.sql(sql).explain() - verifyFrame(spark.sql(sql), 5, 1) - } - - test("create table statement") { - spark.sql("drop table if exists ptab") - spark.sql("create table ptab (col1 varchar(14), col2 bigint, col3 bigint) " + - "using arrow " + - "partitioned by (col1)") - spark.sql("select * from ptab") - } - - test("simple SQL query on parquet file - 1") { - val path = ArrowDataSourceTest.locateResourcePath(parquetFile1) - val frame = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .arrow(path) - frame.createOrReplaceTempView("ptab") - verifyFrame(spark.sql("select * from ptab"), 5, 1) - verifyFrame(spark.sql("select col from ptab"), 5, 1) - verifyFrame(spark.sql("select col from ptab where col is not null or col is null"), - 5, 1) - } - - test("simple SQL query on parquet file - 2") { - val path = ArrowDataSourceTest.locateResourcePath(parquetFile3) - val frame = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .arrow(path) - frame.createOrReplaceTempView("ptab") - val sqlFrame = spark.sql("select * from ptab") - assert( - sqlFrame.schema === - StructType(Seq(StructField("col1", StringType), StructField("col2", LongType)))) - val rows = sqlFrame.collect() - assert(rows(0).get(0) == "apple") - assert(rows(0).get(1) == 100) - assert(rows(1).get(0) == "pear") - assert(rows(1).get(1) == 200) - assert(rows(2).get(0) == "apple") - assert(rows(2).get(1) == 300) - assert(rows.length === 3) - } - - test("simple parquet write") { - val path = ArrowDataSourceTest.locateResourcePath(parquetFile3) - val frame = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .arrow(path) - frame.createOrReplaceTempView("ptab") - val sqlFrame = spark.sql("select * from ptab") - - val writtenPath = FileUtils.getTempDirectory + File.separator + "written.parquet" - sqlFrame.write.mode(SaveMode.Overwrite) - .option(ArrowOptions.KEY_TARGET_FORMAT, "parquet") - .arrow(writtenPath) - - val frame2 = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .arrow(writtenPath) - frame2.createOrReplaceTempView("ptab2") - val sqlFrame2 = spark.sql("select * from ptab2") - - verifyFrame(sqlFrame2, 3, 2) - } - - test("simple SQL query on parquet file with pushed filters") { - val path = ArrowDataSourceTest.locateResourcePath(parquetFile1) - val frame = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .arrow(path) - frame.createOrReplaceTempView("ptab") - spark.sql("select col from ptab where col = 1").explain(true) - val result = spark.sql("select col from ptab where col = 1") // fixme rowcount == 2? - assert( - result.schema === - StructType(Seq(StructField("col", LongType)))) - assert(result.collect().length === 1) - } - - test("ignore unrecognizable types when pushing down filters") { - val path = ArrowDataSourceTest.locateResourcePath(parquetFile2) - val frame = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .arrow(path) - frame.createOrReplaceTempView("ptab") - val rows = spark.sql("select * from ptab where col = 'b'").collect() - assert(rows.length === 1) - } - - ignore("dynamic partition pruning") { - withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", - SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false", - SQLConf.USE_V1_SOURCE_LIST.key -> "arrow", - SQLConf.CBO_ENABLED.key -> "true") { - - var path: String = null - path = ArrowDataSourceTest.locateResourcePath(parquetFile4) - spark.catalog.createTable("df1", path, "arrow") - path = ArrowDataSourceTest.locateResourcePath(parquetFile5) - spark.catalog.createTable("df2", path, "arrow") - - sql("ALTER TABLE df1 RECOVER PARTITIONS") - sql("ALTER TABLE df2 RECOVER PARTITIONS") - - sql("ANALYZE TABLE df1 COMPUTE STATISTICS FOR COLUMNS id") - sql("ANALYZE TABLE df2 COMPUTE STATISTICS FOR COLUMNS id") - - val df = sql("SELECT df1.id, df2.k FROM df1 " + - "JOIN df2 ON df1.k = df2.k AND df2.id < 2") - assert(df.queryExecution.executedPlan.toString().contains("dynamicpruningexpression")) - checkAnswer(df, Row(0, 0) :: Row(1, 1) :: Nil) - } - } - - test("count(*) without group by v2") { - val path = ArrowDataSourceTest.locateResourcePath(parquetFile1) - val frame = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .arrow(path) - frame.createOrReplaceTempView("ptab") - val df = sql("SELECT COUNT(*) FROM ptab") - checkAnswer(df, Row(5) :: Nil) - - } - - test("file descriptor leak") { - val path = ArrowDataSourceTest.locateResourcePath(parquetFile1) - val frame = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .arrow(path) - frame.createOrReplaceTempView("ptab") - - def getFdCount: Long = { - ManagementFactory.getOperatingSystemMXBean - .asInstanceOf[UnixOperatingSystemMXBean] - .getOpenFileDescriptorCount - } - - val initialFdCount = getFdCount - for (_ <- 0 until 100) { - verifyFrame(spark.sql("select * from ptab"), 5, 1) - } - val fdGrowth = getFdCount - initialFdCount - assert(fdGrowth < 100) - } - - test("file descriptor leak - v1") { - val path = ArrowDataSourceTest.locateResourcePath(parquetFile1) - val frame = spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "parquet") - .arrow(path) - frame.createOrReplaceTempView("ptab2") - - def getFdCount: Long = { - ManagementFactory.getOperatingSystemMXBean - .asInstanceOf[UnixOperatingSystemMXBean] - .getOpenFileDescriptorCount - } - - val initialFdCount = getFdCount - for (_ <- 0 until 100) { - verifyFrame(spark.sql("select * from ptab2"), 5, 1) - } - val fdGrowth = getFdCount - initialFdCount - assert(fdGrowth < 100) - } - - private val csvFile1 = "people.csv" - private val csvFile2 = "example.csv" - private val csvFile3 = "example-tab.csv" - - ignore("read csv file without specifying original format") { - // not implemented - verifyFrame(spark.read.format("arrow") - .load(ArrowDataSourceTest.locateResourcePath(csvFile1)), 1, 2) - } - - test("read csv file") { - val path = ArrowDataSourceTest.locateResourcePath(csvFile1) - verifyFrame( - spark.read - .format("arrow") - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "csv") - .load(path), 2, 3) - } - - test("read csv file 2") { - val path = ArrowDataSourceTest.locateResourcePath(csvFile2) - verifyFrame( - spark.read - .format("arrow") - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "csv") - .load(path), 34, 9) - } - - test("read csv file 3 - tab separated") { - val path = ArrowDataSourceTest.locateResourcePath(csvFile3) - verifyFrame( - spark.read - .format("arrow") - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "csv") - .option("delimiter", "\t") - .load(path), 34, 9) - } - - test("read csv file - programmatic API ") { - val path = ArrowDataSourceTest.locateResourcePath(csvFile1) - verifyFrame( - spark.read - .option(ArrowOptions.KEY_ORIGINAL_FORMAT, "csv") - .arrow(path), 2, 3) - } - - def verifyFrame(frame: DataFrame, rowCount: Int, columnCount: Int): Unit = { - assert(frame.schema.length === columnCount) - assert(frame.collect().length === rowCount) - } - - def verifyCsv(frame: DataFrame): Unit = { - // todo assert something - } - - def verifyParquet(frame: DataFrame): Unit = { - verifyFrame(frame, 5, 1) - } - - def delete(path: String): Unit = { - FileUtils.forceDelete(new File(path)) - } - - def closeAllocators(): Unit = { - SparkMemoryUtils.contextAllocator().close() - } -} - -object ArrowDataSourceTest { - private def locateResourcePath(resource: String): String = { - classOf[ArrowDataSourceTest].getClassLoader.getResource("") - .getPath.concat(File.separator).concat(resource) - } -} diff --git a/native-sql-engine/core/pom.xml b/native-sql-engine/core/pom.xml index 1786ff874699..c059b6e5cb16 100644 --- a/native-sql-engine/core/pom.xml +++ b/native-sql-engine/core/pom.xml @@ -140,16 +140,28 @@ - spark-arrow-datasource-common - com.intel.oap - ${project.version} - provided - - - spark-arrow-datasource-standard - com.intel.oap - ${project.version} - provided + org.apache.arrow + arrow-dataset + ${arrow.version} + + + io.netty + netty-common + + + io.netty + netty-buffer + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-annotations + + + compile org.scalacheck diff --git a/arrow-data-source/common/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryConsumer.java b/native-sql-engine/core/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryConsumer.java similarity index 100% rename from arrow-data-source/common/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryConsumer.java rename to native-sql-engine/core/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryConsumer.java diff --git a/arrow-data-source/common/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryMetrics.java b/native-sql-engine/core/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryMetrics.java similarity index 100% rename from arrow-data-source/common/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryMetrics.java rename to native-sql-engine/core/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryMetrics.java diff --git a/native-sql-engine/core/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/SparkManagedAllocationListener.java b/native-sql-engine/core/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/SparkManagedAllocationListener.java new file mode 100644 index 000000000000..14ed5a7da608 --- /dev/null +++ b/native-sql-engine/core/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/SparkManagedAllocationListener.java @@ -0,0 +1,81 @@ +/* + * 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 com.intel.oap.spark.sql.execution.datasources.v2.arrow; + +import org.apache.arrow.memory.AllocationListener; + +public class SparkManagedAllocationListener implements AllocationListener { + public static long BLOCK_SIZE = 8L * 1024 * 1024; // 8MB per block + + private final NativeSQLMemoryConsumer consumer; + private final NativeSQLMemoryMetrics metrics; + + private long bytesReserved = 0L; + private long blocksReserved = 0L; + + public SparkManagedAllocationListener(NativeSQLMemoryConsumer consumer, NativeSQLMemoryMetrics metrics) { + this.consumer = consumer; + this.metrics = metrics; + } + + @Override + public void onPreAllocation(long size) { + long requiredBlocks = updateReservation(size); + if (requiredBlocks < 0) { + throw new IllegalStateException(); + } + if (requiredBlocks == 0) { + return; + } + long toBeAcquired = requiredBlocks * BLOCK_SIZE; + consumer.acquire(toBeAcquired); + metrics.inc(toBeAcquired); + } + + @Override + public void onRelease(long size) { + long requiredBlocks = updateReservation(-size); + if (requiredBlocks > 0) { + throw new IllegalStateException(); + } + if (requiredBlocks == 0) { + return; + } + long toBeReleased = -requiredBlocks * BLOCK_SIZE; + consumer.free(toBeReleased); + metrics.inc(-toBeReleased); + } + + public long updateReservation(long bytesToAdd) { + synchronized (this) { + long newBytesReserved = bytesReserved + bytesToAdd; + final long newBlocksReserved; + // ceiling + if (newBytesReserved == 0L) { + // 0 is the special case in ceiling algorithm + newBlocksReserved = 0L; + } else { + newBlocksReserved = (newBytesReserved - 1L) / BLOCK_SIZE + 1L; + } + long requiredBlocks = newBlocksReserved - blocksReserved; + bytesReserved = newBytesReserved; + blocksReserved = newBlocksReserved; + return requiredBlocks; + } + } +} diff --git a/arrow-data-source/common/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/SparkManagedReservationListener.java b/native-sql-engine/core/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/SparkManagedReservationListener.java similarity index 100% rename from arrow-data-source/common/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/SparkManagedReservationListener.java rename to native-sql-engine/core/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/SparkManagedReservationListener.java diff --git a/arrow-data-source/common/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/Spiller.java b/native-sql-engine/core/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/Spiller.java similarity index 100% rename from arrow-data-source/common/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/Spiller.java rename to native-sql-engine/core/src/main/java/com/intel/oap/spark/sql/execution/datasources/v2/arrow/Spiller.java diff --git a/arrow-data-source/common/src/main/java/com/intel/oap/vectorized/ArrowWritableColumnVector.java b/native-sql-engine/core/src/main/java/com/intel/oap/vectorized/ArrowWritableColumnVector.java similarity index 100% rename from arrow-data-source/common/src/main/java/com/intel/oap/vectorized/ArrowWritableColumnVector.java rename to native-sql-engine/core/src/main/java/com/intel/oap/vectorized/ArrowWritableColumnVector.java diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/ArrowColumnarToRowExec.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/ArrowColumnarToRowExec.scala index b13f623fac9d..cb3a120e2ae5 100644 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/ArrowColumnarToRowExec.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/ArrowColumnarToRowExec.scala @@ -20,7 +20,6 @@ package com.intel.oap.execution import com.intel.oap.expression.ConverterUtils import com.intel.oap.vectorized.{ArrowColumnarToRowJniWrapper, ArrowWritableColumnVector} import org.apache.arrow.vector.types.pojo.{Field, Schema} - import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow} diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/BasicPhysicalOperatorTransformer.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/BasicPhysicalOperatorTransformer.scala index cb4effdae041..908bedc8486c 100644 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/BasicPhysicalOperatorTransformer.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/BasicPhysicalOperatorTransformer.scala @@ -38,7 +38,6 @@ import com.google.common.collect.Lists import com.intel.oap.GazellePluginConfig import com.intel.oap.substrait.expression.ExpressionNode import com.intel.oap.substrait.rel.{RelBuilder, RelNode} -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils; case class ConditionProjectExecTransformer( condition: Expression, diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/BatchScanExecTransformer.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/BatchScanExecTransformer.scala index 8ed911473e20..ae5828fb8ab4 100644 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/BatchScanExecTransformer.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/BatchScanExecTransformer.scala @@ -25,10 +25,9 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory, Scan} import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, FileScan} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} -import com.intel.oap.spark.sql.execution.datasources.v2.arrow.ArrowScan import com.intel.oap.substrait.`type`.TypeBuiler import com.intel.oap.substrait.expression.{ExpressionBuilder, ExpressionNode} import org.apache.spark.sql.execution.datasources.FilePartition @@ -38,12 +37,8 @@ import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan class BatchScanExecTransformer(output: Seq[AttributeReference], @transient scan: Scan) extends BatchScanExec(output, scan) with TransformSupport { val tmpDir: String = GazellePluginConfig.getConf.tmpFile - val filterExprs: Seq[Expression] = if (scan.isInstanceOf[ParquetScan]) { - scan.asInstanceOf[ParquetScan].dataFilters - } else if (scan.isInstanceOf[OrcScan]) { - scan.asInstanceOf[OrcScan].dataFilters - } else if (scan.isInstanceOf[ArrowScan]) { - scan.asInstanceOf[ArrowScan].dataFilters + val filterExprs: Seq[Expression] = if (scan.isInstanceOf[FileScan]) { + scan.asInstanceOf[FileScan].dataFilters } else { throw new UnsupportedOperationException(s"${scan.getClass.toString} is not supported") } @@ -57,18 +52,7 @@ class BatchScanExecTransformer(output: Seq[AttributeReference], @transient scan: "inputSize" -> SQLMetrics.createSizeMetric(sparkContext, "input size in bytes")) override def doExecuteColumnar(): RDD[ColumnarBatch] = { - val numOutputRows = longMetric("numOutputRows") - val numInputBatches = longMetric("numInputBatches") - val numOutputBatches = longMetric("numOutputBatches") - val scanTime = longMetric("scanTime") - val inputSize = longMetric("inputSize") - val inputColumnarRDD = - new ColumnarDataSourceRDD(sparkContext, partitions, readerFactory, true, scanTime, numInputBatches, inputSize, tmpDir) - inputColumnarRDD.map { r => - numOutputRows += r.numRows() - numOutputBatches += 1 - r - } + throw new UnsupportedOperationException(s"This operator doesn't support doExecuteColumnar().") } override def canEqual(other: Any): Boolean = other.isInstanceOf[BatchScanExecTransformer] diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/BroadcastHashJoinExecTransformer.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/BroadcastHashJoinExecTransformer.scala index c1606f3f8ed4..b474453b411e 100644 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/BroadcastHashJoinExecTransformer.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/BroadcastHashJoinExecTransformer.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils import org.apache.spark.sql.execution.joins.{BaseJoinExec, HashJoin, ShuffledJoin} import org.apache.spark.sql.execution.joins.HashedRelationInfo import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/CoalesceBatchesExec.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/CoalesceBatchesExec.scala index 476c48e6b62b..b1ba0bdf3470 100644 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/CoalesceBatchesExec.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/CoalesceBatchesExec.scala @@ -20,22 +20,20 @@ package com.intel.oap.execution import com.intel.oap.expression.ConverterUtils import com.intel.oap.vectorized.ArrowWritableColumnVector import com.intel.oap.vectorized.CloseableColumnBatchIterator - import org.apache.arrow.vector.util.VectorBatchAppender import org.apache.arrow.memory.{BufferAllocator, RootAllocator} -import org.apache.arrow.vector.types.pojo.Schema; - +import org.apache.arrow.vector.types.pojo.Schema import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils -import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} -import org.apache.spark.sql.types.{StructType, StructField} -import org.apache.spark.sql.util.ArrowUtils; +import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} +import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.util.ArrowUtils import scala.collection.mutable.ListBuffer import scala.collection.JavaConverters._ diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/ColumnarDataSourceRDD.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/ColumnarDataSourceRDD.scala deleted file mode 100644 index 0187e7577057..000000000000 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/ColumnarDataSourceRDD.scala +++ /dev/null @@ -1,137 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.intel.oap.execution - -import com.intel.oap.GazellePluginConfig -import com.intel.oap.vectorized._ -import org.apache.spark._ -import org.apache.spark.rdd.RDD -import org.apache.spark.util._ -import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} -import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} -import org.apache.spark.sql.execution.datasources.v2.VectorizedFilePartitionReaderHandler -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils -import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetPartitionReaderFactory -import org.apache.spark.sql.util.OASPackageBridge._ - -class DataSourceRDDPartition(val index: Int, val inputPartition: InputPartition) - extends Partition - with Serializable - -// TODO: we should have 2 RDDs: an RDD[InternalRow] for row-based scan, an `RDD[ColumnarBatch]` for -// columnar scan. -class ColumnarDataSourceRDD( - sc: SparkContext, - @transient private val inputPartitions: Seq[InputPartition], - partitionReaderFactory: PartitionReaderFactory, - columnarReads: Boolean, - scanTime: SQLMetric, - numInputBatches: SQLMetric, - inputSize: SQLMetric, - tmp_dir: String) - extends RDD[ColumnarBatch](sc, Nil) { - val numaBindingInfo = GazellePluginConfig.getConf.numaBindingInfo - - override protected def getPartitions: Array[Partition] = { - inputPartitions.zipWithIndex.map { - case (inputPartition, index) => new DataSourceRDDPartition(index, inputPartition) - }.toArray - } - - private def castPartition(split: Partition): DataSourceRDDPartition = split match { - case p: DataSourceRDDPartition => p - case _ => throw new SparkException(s"[BUG] Not a DataSourceRDDPartition: $split") - } - - override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { - ExecutorManager.tryTaskSet(numaBindingInfo) - val inputPartition = castPartition(split).inputPartition - inputPartition match { - case p: FilePartition => - p.files.foreach { f => inputSize += f.length } - case _ => - } - val reader = if (columnarReads) { - partitionReaderFactory match { - case factory: ParquetPartitionReaderFactory => - VectorizedFilePartitionReaderHandler.get(inputPartition, factory, tmp_dir) - case _ => partitionReaderFactory.createColumnarReader(inputPartition) - } - } else { - partitionReaderFactory.createReader(inputPartition) - } - - val rddId = this - SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit](_ => reader.close()) - val iter = new Iterator[Any] { - private val inputMetrics = TaskContext.get().taskMetrics().inputMetrics - - private[this] var valuePrepared = false - - override def hasNext: Boolean = { - if (!valuePrepared) { - try { - val beforeScan = System.nanoTime() - valuePrepared = reader.next() - numInputBatches += 1 - scanTime += (System.nanoTime() - beforeScan) / (1000 * 1000) - } catch { - case e: Throwable => - val errmsg = e.getStackTrace.mkString("\n") - logError(s"hasNext got exception: $errmsg") - valuePrepared = false - } - } - valuePrepared - } - - override def next(): Any = { - if (!hasNext) { - throw new java.util.NoSuchElementException("End of stream") - } - valuePrepared = false - val value = reader.get() - val bytes: Long = value match { - case batch: ColumnarBatch => - (0 until batch.numCols()).map { i => - val vector = Option(batch.column(i)) - vector.map { - case av: ArrowWritableColumnVector => - av.getValueVector.getBufferSize.toLong - case _ => 0L - }.sum - }.sum - case _ => 0L - } - inputMetrics.bridgeIncBytesRead(bytes) - value - } - } - val closeableColumnarBatchIterator = new CloseableColumnBatchIterator( - iter.asInstanceOf[Iterator[ColumnarBatch]]) - // TODO: SPARK-25083 remove the type erasure hack in data source scan - new InterruptibleIterator(context, closeableColumnarBatchIterator) - } - - override def getPreferredLocations(split: Partition): Seq[String] = { - castPartition(split).inputPartition.preferredLocations() - } - -} diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/ExpandExecTransformer.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/ExpandExecTransformer.scala index e6c7657b06cc..622af76dfe9f 100644 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/ExpandExecTransformer.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/ExpandExecTransformer.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartit import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.TaskContext -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils import org.apache.spark.sql.types.DecimalType case class ExpandExecTransformer( diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/HashAggregateExecTransformer.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/HashAggregateExecTransformer.scala index a80c1a7d3a54..a322c97b0064 100644 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/HashAggregateExecTransformer.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/HashAggregateExecTransformer.scala @@ -48,7 +48,6 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.aggregate._ -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.vectorized.MutableColumnarRow import org.apache.spark.sql.internal.SQLConf diff --git a/arrow-data-source/common/src/main/scala/com/intel/oap/sql/execution/RowToArrowColumnarExec.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/RowToArrowColumnarExec.scala similarity index 99% rename from arrow-data-source/common/src/main/scala/com/intel/oap/sql/execution/RowToArrowColumnarExec.scala rename to native-sql-engine/core/src/main/scala/com/intel/oap/execution/RowToArrowColumnarExec.scala index b9b58fcb9c90..9bd55f729866 100644 --- a/arrow-data-source/common/src/main/scala/com/intel/oap/sql/execution/RowToArrowColumnarExec.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/RowToArrowColumnarExec.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package com.intel.oap.sql.execution +package com.intel.oap.execution import java.util.concurrent.TimeUnit._ diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/ShuffledHashJoinExecTransformer.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/ShuffledHashJoinExecTransformer.scala index 5f3ee46e83c9..af4c41f4a8f8 100644 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/ShuffledHashJoinExecTransformer.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/ShuffledHashJoinExecTransformer.scala @@ -49,7 +49,6 @@ import org.apache.arrow.memory.ArrowBuf import com.google.common.collect.Lists import com.intel.oap.expression._ import com.intel.oap.vectorized.ExpressionEvaluator -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.execution.joins.{HashJoin,ShuffledJoin,BaseJoinExec} diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/SortExecTransformer.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/SortExecTransformer.scala index 9b2fb8cd5882..2932fd0a6cdf 100644 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/SortExecTransformer.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/SortExecTransformer.scala @@ -39,7 +39,6 @@ import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReference import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils import org.apache.spark.util.{ExecutorManager, UserAddedJarUtils, Utils} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/SortMergeJoinExecTransformer.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/SortMergeJoinExecTransformer.scala index d04d81f1e0e3..9aa5e35130f6 100644 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/SortMergeJoinExecTransformer.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/SortMergeJoinExecTransformer.scala @@ -50,7 +50,6 @@ import org.apache.arrow.memory.ArrowBuf import com.google.common.collect.Lists import com.intel.oap.expression._ import com.intel.oap.vectorized.ExpressionEvaluator -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.types.DecimalType diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/WholestageColumnarRDD.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/WholestageColumnarRDD.scala index 0af1de815677..461d35edbd13 100644 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/WholestageColumnarRDD.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/WholestageColumnarRDD.scala @@ -32,7 +32,6 @@ import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFacto import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.datasources.FilePartition import org.apache.spark.sql.execution.datasources.v2.VectorizedFilePartitionReaderHandler -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetPartitionReaderFactory import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.util.ArrowUtils diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/WindowExecTransformer.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/WindowExecTransformer.scala index 3430a2480349..f9e3028a573b 100644 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/execution/WindowExecTransformer.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/execution/WindowExecTransformer.scala @@ -36,7 +36,6 @@ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistrib import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.{SortExec, SparkPlan} -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.window.WindowExecBase import org.apache.spark.sql.internal.SQLConf @@ -49,7 +48,6 @@ import scala.collection.immutable.Stream.Empty import scala.collection.mutable.ListBuffer import scala.util.Random -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkSchemaUtils import util.control.Breaks._ case class WindowExecTransformer(windowExpression: Seq[NamedExpression], diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/expression/CodeGeneration.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/expression/CodeGeneration.scala index d68a40e3ccf9..bb5882b3fbc0 100644 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/expression/CodeGeneration.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/expression/CodeGeneration.scala @@ -21,7 +21,6 @@ import org.apache.arrow.vector.Float4Vector import org.apache.arrow.vector.IntVector import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, TimeUnit} import org.apache.arrow.vector.types.pojo.ArrowType - import org.apache.spark.sql.execution.datasources.v2.arrow.SparkSchemaUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/expression/ConverterUtils.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/expression/ConverterUtils.scala index eaa7606ba9a6..fca4342fa30e 100644 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/expression/ConverterUtils.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/expression/ConverterUtils.scala @@ -62,8 +62,7 @@ import org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision} import org.apache.spark.sql.catalyst.util.DateTimeConstants import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_SECOND -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkSchemaUtils -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkVectorUtils +import org.apache.spark.sql.execution.datasources.v2.arrow.{SparkSchemaUtils, SparkVectorUtils} object ConverterUtils extends Logging { def calcuateEstimatedSize(columnarBatch: ColumnarBatch): Long = { diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/expression/UnaryOperatorTransformer.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/expression/UnaryOperatorTransformer.scala index f092d801ad52..3411966013c0 100644 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/expression/UnaryOperatorTransformer.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/expression/UnaryOperatorTransformer.scala @@ -36,7 +36,6 @@ import com.intel.oap.substrait.`type`.TypeBuiler import com.intel.oap.substrait.expression.{ExpressionBuilder, ExpressionNode} import org.apache.arrow.vector.types.TimeUnit import org.apache.spark.sql.catalyst.util.DateTimeConstants -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkSchemaUtils /** * A version of add that supports columnar processing for longs. diff --git a/native-sql-engine/core/src/main/scala/com/intel/oap/extension/ColumnarOverrides.scala b/native-sql-engine/core/src/main/scala/com/intel/oap/extension/ColumnarOverrides.scala index 557c987906c5..41661762e3b1 100644 --- a/native-sql-engine/core/src/main/scala/com/intel/oap/extension/ColumnarOverrides.scala +++ b/native-sql-engine/core/src/main/scala/com/intel/oap/extension/ColumnarOverrides.scala @@ -19,7 +19,6 @@ package com.intel.oap import com.intel.oap.execution._ import com.intel.oap.extension.columnar.{RowGuard, TransformGuardRule} -import com.intel.oap.sql.execution.RowToArrowColumnarExec import org.apache.spark.internal.config._ import org.apache.spark.internal.Logging import org.apache.spark.sql.{SparkSession, SparkSessionExtensions} diff --git a/arrow-data-source/common/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala b/native-sql-engine/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala similarity index 98% rename from arrow-data-source/common/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala rename to native-sql-engine/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala index a5067098eec1..a6dd4dc707e2 100644 --- a/arrow-data-source/common/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala +++ b/native-sql-engine/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala @@ -15,23 +15,23 @@ * limitations under the License. */ + package org.apache.spark.sql.execution.datasources.v2.arrow import java.io.PrintWriter import java.util import java.util.UUID -import scala.collection.JavaConverters._ - -import com.intel.oap.spark.sql.execution.datasources.v2.arrow._ +import com.intel.oap.spark.sql.execution.datasources.v2.arrow.{NativeSQLMemoryConsumer, NativeSQLMemoryMetrics, SparkManagedAllocationListener, SparkManagedReservationListener, Spiller} import com.sun.xml.internal.messaging.saaj.util.ByteOutputStream + +import scala.collection.JavaConverters._ import org.apache.arrow.dataset.jni.NativeMemoryPool import org.apache.arrow.memory.AllocationListener import org.apache.arrow.memory.BufferAllocator import org.apache.arrow.memory.MemoryChunkCleaner import org.apache.arrow.memory.MemoryChunkManager import org.apache.arrow.memory.RootAllocator - import org.apache.spark.SparkEnv import org.apache.spark.TaskContext import org.apache.spark.internal.Logging diff --git a/arrow-data-source/common/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkSchemaUtils.scala b/native-sql-engine/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkSchemaUtils.scala similarity index 100% rename from arrow-data-source/common/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkSchemaUtils.scala rename to native-sql-engine/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkSchemaUtils.scala diff --git a/arrow-data-source/common/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkVectorUtils.scala b/native-sql-engine/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkVectorUtils.scala similarity index 100% rename from arrow-data-source/common/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkVectorUtils.scala rename to native-sql-engine/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkVectorUtils.scala diff --git a/native-sql-engine/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExecTransformer.scala b/native-sql-engine/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExecTransformer.scala index 19e3723ea0c9..d9bc26df0179 100644 --- a/native-sql-engine/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExecTransformer.scala +++ b/native-sql-engine/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExecTransformer.scala @@ -36,7 +36,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.python.EvalPythonExec -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnVector, ColumnarBatch} import org.apache.spark.sql.types.{DataType, StructField, StructType} diff --git a/pom.xml b/pom.xml index 870a6e5f6bbb..95692576b804 100644 --- a/pom.xml +++ b/pom.xml @@ -30,7 +30,6 @@ - arrow-data-source native-sql-engine/core shims @@ -196,6 +195,14 @@ org.apache.curator curator-recipes + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + diff --git a/shims/pom.xml b/shims/pom.xml index fd185052c3f1..aa1ff1029abe 100644 --- a/shims/pom.xml +++ b/shims/pom.xml @@ -44,23 +44,11 @@ - org.slf4j - slf4j-log4j12 - 1.7.30 - test - - - log4j - log4j - 1.2.17 - test - - - org.apache.hadoop - hadoop-client - ${hadoop.version} - test - + org.apache.hadoop + hadoop-client + ${hadoop.version} + test +