diff --git a/.github/workflows/unittests.yml b/.github/workflows/unittests.yml index 50cd992949b9..21cd8601c314 100644 --- a/.github/workflows/unittests.yml +++ b/.github/workflows/unittests.yml @@ -58,7 +58,7 @@ jobs: - name: Install OAP optimized Arrow (C++ libs) run: | bash tools/build_arrow.sh --build_arrow=ON - mvn clean package -DskipTests -Dcheckstyle.skip -Dbuild_cpp=ON -Dbuild_gazelle_cpp=ON -Darrow_root=/tmp/arrow_install.8 + mvn clean package -Pbackends-gazelle -DskipTests -Dcheckstyle.skip -Dbuild_cpp=ON -Dbuild_gazelle_cpp=ON -Darrow_root=/tmp/arrow_install.8 - name: Run unit tests run: | cd cpp/ diff --git a/backends-clickhouse/pom.xml b/backends-clickhouse/pom.xml new file mode 100644 index 000000000000..0216fdd5d4f6 --- /dev/null +++ b/backends-clickhouse/pom.xml @@ -0,0 +1,287 @@ + + + + gluten-parent + io.glutenproject + 1.0.0-snapshot + + 4.0.0 + + backends-clickhouse + jar + Gluten-Backends-ClickHouse + + + 8 + 8 + ${project.basedir}/src/main/resources + + + + + org.apache.spark + spark-sql_${scala.binary.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + provided + + + org.apache.spark + spark-catalyst_${scala.binary.version} + provided + + + org.apache.spark + spark-sql_${scala.binary.version} + provided + + + org.apache.spark + spark-hive-thriftserver_${scala.binary.version} + provided + + + io.delta + delta-core_${scala.binary.version} + provided + + + io.glutenproject + spark-sql-columnar-shims-common + ${project.version} + compile + + + io.glutenproject + gluten-jvm + ${project.version} + compile + + + org.scalacheck + scalacheck_${scala.binary.version} + 1.13.5 + test + + + org.scala-lang + scala-library + ${scala.version} + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.mockito + mockito-core + 2.23.4 + test + + + junit + junit + 4.12 + test + + + com.fasterxml.jackson.core + jackson-core + 2.10.0 + provided + + + com.fasterxml.jackson.core + jackson-annotations + 2.10.0 + provided + + + com.fasterxml.jackson.core + jackson-databind + 2.10.0 + provided + + + org.scalatestplus + scalatestplus-mockito_2.12 + 1.0.0-M2 + test + + + org.scalatestplus + scalatestplus-scalacheck_2.12 + 3.1.0.0-RC2 + test + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + provided + + + commons-io + commons-io + 2.11.0 + provided + + + + + + + ${resource.dir} + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-resources-plugin + 3.0.1 + + + net.alchim31.maven + scala-maven-plugin + 4.3.0 + + ${scala.recompile.mode} + + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.8.0 + + 1.8 + 1.8 + + + + compile + + compile + + + + + + maven-assembly-plugin + 3.3.0 + + ${jar.assembly.name.prefix}-${project.version} + + jar-with-dependencies + + + + + make-assembly + package + + single + + + + + + org.scalastyle + scalastyle-maven-plugin + 1.0.0 + + false + false + true + false + ${project.basedir}/src/main/scala + ${project.basedir}/src/test/scala + ${project.basedir}/scalastyle-config.xml + ${project.basedir}/scalastyle-output.xml + UTF-8 + + + + + check + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 3.1.1 + + false + true + + ${project.basedir}/src/main/java + ${project.basedir}/src/main/scala + + + ${project.basedir}/src/test/java + + ${project.basedir}/../dev/checkstyle.xml + ${project.basedir}/target/checkstyle-output.xml + UTF-8 + UTF-8 + + + + com.puppycrawl.tools + checkstyle + 8.29 + + + + + + check + + + + + + org.scalatest + scalatest-maven-plugin + + true + . + + + + test + + test + + + + + + + \ No newline at end of file diff --git a/backends-clickhouse/scalastyle-config.xml b/backends-clickhouse/scalastyle-config.xml new file mode 100644 index 000000000000..9585785835d6 --- /dev/null +++ b/backends-clickhouse/scalastyle-config.xml @@ -0,0 +1,433 @@ + + + + + Scalastyle standard configuration + + + + + + + + + + + + + + + + + + + + + + + + true + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ARROW, EQUALS, ELSE, TRY, CATCH, FINALLY, LARROW, RARROW + + + + + + ARROW, EQUALS, COMMA, COLON, IF, ELSE, DO, WHILE, FOR, MATCH, TRY, CATCH, FINALLY, LARROW, RARROW + + + + + + + + + ^FunSuite[A-Za-z]*$ + Tests must extend org.apache.spark.SparkFunSuite instead. + + + + + ^println$ + + + + + spark(.sqlContext)?.sparkContext.hadoopConfiguration + + + + + @VisibleForTesting + + + + + Runtime\.getRuntime\.addShutdownHook + + + + + mutable\.SynchronizedBuffer + + + + + Class\.forName + + + + + Await\.result + + + + + Await\.ready + + + + + (\.toUpperCase|\.toLowerCase)(?!(\(|\(Locale.ROOT\))) + + + + + throw new \w+Error\( + + + + + + JavaConversions + Instead of importing implicits in scala.collection.JavaConversions._, import + scala.collection.JavaConverters._ and use .asScala / .asJava methods + + + + org\.apache\.commons\.lang\. + Use Commons Lang 3 classes (package org.apache.commons.lang3.*) instead + of Commons Lang 2 (package org.apache.commons.lang.*) + + + + scala\.concurrent\.ExecutionContext\.Implicits\.global + User queries can use global thread pool, causing starvation and eventual OOM. + Thus, Spark-internal APIs should not use this thread pool + + + + FileSystem.get\([a-zA-Z_$][a-zA-Z_$0-9]*\) + + + + + extractOpt + Use jsonOption(x).map(.extract[T]) instead of .extractOpt[T], as the latter + is slower. + + + + + java,scala,3rdParty,spark + javax?\..* + scala\..* + (?!org\.apache\.spark\.).* + org\.apache\.spark\..* + + + + + + COMMA + + + + + + \)\{ + + + + + (?m)^(\s*)/[*][*].*$(\r|)\n^\1 [*] + Use Javadoc style indentation for multiline comments + + + + case[^\n>]*=>\s*\{ + Omit braces in case clauses. + + + + new (java\.lang\.)?(Byte|Integer|Long|Short)\( + Use static factory 'valueOf' or 'parseXXX' instead of the deprecated constructors. + + + + + + + + + + + + + + + Please use Apache Log4j 2 instead. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 800> + + + + + 30 + + + + + 10 + + + + + 50 + + + + + + + + + + + -1,0,1,2,3 + + + + Objects.toStringHelper + Avoid using Object.toStringHelper. Use ToStringBuilder instead. + + diff --git a/backends-clickhouse/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java b/backends-clickhouse/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java new file mode 100644 index 000000000000..d291219fe9bb --- /dev/null +++ b/backends-clickhouse/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.execution; + +import io.glutenproject.vectorized.CHColumnVector; +import org.apache.spark.sql.vectorized.ColumnarBatch; + +import java.util.Iterator; + +public class ColumnarNativeIterator extends AbstractColumnarNativeIterator { + + public ColumnarNativeIterator(Iterator delegated) { + super(delegated); + } + + @Override + public byte[] next() { + ColumnarBatch dep_cb = nextBatch; + if (dep_cb.numRows() > 0) { + CHColumnVector col = (CHColumnVector) dep_cb.column(0); + return longtoBytes(col.getBlockAddress()); + } else { + throw new IllegalStateException(); + } + } + + private static byte[] longtoBytes(long data) { + return new byte[]{ + (byte) ((data >> 56) & 0xff), + (byte) ((data >> 48) & 0xff), + (byte) ((data >> 40) & 0xff), + (byte) ((data >> 32) & 0xff), + (byte) ((data >> 24) & 0xff), + (byte) ((data >> 16) & 0xff), + (byte) ((data >> 8) & 0xff), + (byte) ((data >> 0) & 0xff), + }; + } +} diff --git a/backends-clickhouse/src/main/java/io/glutenproject/vectorized/BatchIterator.java b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/BatchIterator.java new file mode 100644 index 000000000000..5af1458979a7 --- /dev/null +++ b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/BatchIterator.java @@ -0,0 +1,68 @@ +/* + * 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 io.glutenproject.vectorized; + +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarBatch; + +import java.io.IOException; + +public class BatchIterator extends AbstractBatchIterator { + private native boolean nativeHasNext(long nativeHandler); + private native byte[] nativeNext(long nativeHandler); + private native long nativeCHNext(long nativeHandler); + private native void nativeClose(long nativeHandler); + private native MetricsObject nativeFetchMetrics(long nativeHandler); + + public BatchIterator() throws IOException {} + + public BatchIterator(long instance_id) throws IOException { + super(instance_id); + } + + @Override + public boolean hasNextInternal() throws IOException { + return nativeHasNext(nativeHandler); + } + + @Override + public ColumnarBatch nextInternal() throws IOException { + long block = nativeCHNext(nativeHandler); + CHNativeBlock nativeBlock = new CHNativeBlock(block); + int cols = nativeBlock.numColumns(); + ColumnVector[] columnVectors = new ColumnVector[cols]; + for (int i = 0; i < cols; i++) { + columnVectors[i] = new CHColumnVector(DataType.fromDDL(nativeBlock.getTypeByPosition(i)), block, i); + } + return new ColumnarBatch(columnVectors, nativeBlock.numRows()); + } + + @Override + public MetricsObject getMetricsInternal() throws IOException, ClassNotFoundException { + return nativeFetchMetrics(nativeHandler); + } + + @Override + public void closeInternal() { + if (!closed) { + nativeClose(nativeHandler); + closed = true; + } + } +} diff --git a/jvm/src/main/java/io/glutenproject/vectorized/CHCoalesceOperator.java b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHCoalesceOperator.java similarity index 65% rename from jvm/src/main/java/io/glutenproject/vectorized/CHCoalesceOperator.java rename to backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHCoalesceOperator.java index 29b41a4d8493..6e4a26df696d 100644 --- a/jvm/src/main/java/io/glutenproject/vectorized/CHCoalesceOperator.java +++ b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHCoalesceOperator.java @@ -1,3 +1,20 @@ +/* + * 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 io.glutenproject.vectorized; import org.apache.spark.sql.vectorized.ColumnarBatch; diff --git a/jvm/src/main/java/io/glutenproject/vectorized/CHColumnVector.java b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHColumnVector.java similarity index 100% rename from jvm/src/main/java/io/glutenproject/vectorized/CHColumnVector.java rename to backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHColumnVector.java diff --git a/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHNativeBlock.java b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHNativeBlock.java new file mode 100644 index 000000000000..8e2e49a80b81 --- /dev/null +++ b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHNativeBlock.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.vectorized; + +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarBatch; + +import java.util.Optional; + +public class CHNativeBlock { + private long blockAddress; + + public CHNativeBlock(long blockAddress) { + this.blockAddress = blockAddress; + } + + public static Optional fromColumnarBatch(ColumnarBatch batch) { + if (batch.numCols() == 0 || !(batch.column(0) instanceof CHColumnVector)) { + return Optional.empty(); + } + CHColumnVector columnVector = (CHColumnVector) batch.column(0); + return Optional.of(new CHNativeBlock(columnVector.getBlockAddress())); + } + + private native int nativeNumRows(long blockAddress); + + public int numRows() { + return nativeNumRows(blockAddress); + } + + ; + + private native int nativeNumColumns(long blockAddress); + + public int numColumns() { + return nativeNumColumns(blockAddress); + } + + private native String nativeColumnType(long blockAddress, int position); + + public String getTypeByPosition(int position) { + return nativeColumnType(blockAddress, position); + } + + private native long nativeTotalBytes(long blockAddress); + + public long totalBytes() { + return nativeTotalBytes(blockAddress); + } + + public native void nativeClose(long blockAddress); + + public void close() { + if (blockAddress != 0) { + nativeClose(blockAddress); + blockAddress = 0; + } + } + + public ColumnarBatch toColumnarBatch() { + ColumnVector[] vectors = new ColumnVector[numColumns()]; + for (int i = 0; i < numColumns(); i++) { + vectors[i] = new CHColumnVector(DataType.fromDDL(getTypeByPosition(i)), blockAddress, i); + } + int numRows = 0; + if (numColumns() != 0) { + numRows = numRows(); + } + return new ColumnarBatch(vectors, numRows); + } +} diff --git a/jvm/src/main/java/io/glutenproject/vectorized/CHShuffleSplitterJniWrapper.java b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHShuffleSplitterJniWrapper.java similarity index 100% rename from jvm/src/main/java/io/glutenproject/vectorized/CHShuffleSplitterJniWrapper.java rename to backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHShuffleSplitterJniWrapper.java diff --git a/jvm/src/main/java/io/glutenproject/vectorized/CHStreamReader.java b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHStreamReader.java similarity index 52% rename from jvm/src/main/java/io/glutenproject/vectorized/CHStreamReader.java rename to backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHStreamReader.java index 0945d7848a30..3273f1b68cd1 100644 --- a/jvm/src/main/java/io/glutenproject/vectorized/CHStreamReader.java +++ b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHStreamReader.java @@ -14,34 +14,35 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.glutenproject.vectorized; import java.io.InputStream; -public class CHStreamReader implements AutoCloseable{ - private long nativeShuffleReader; - private final InputStream inputStream; +public class CHStreamReader implements AutoCloseable { + private long nativeShuffleReader; + private final InputStream inputStream; - private static native long createNativeShuffleReader (InputStream inputStream); + private static native long createNativeShuffleReader(InputStream inputStream); - public CHStreamReader(InputStream inputStream) { - this.inputStream = inputStream; - nativeShuffleReader = createNativeShuffleReader(this.inputStream); - } + public CHStreamReader(InputStream inputStream) { + this.inputStream = inputStream; + nativeShuffleReader = createNativeShuffleReader(this.inputStream); + } - private native long nativeNext(long nativeShuffleReader); + private native long nativeNext(long nativeShuffleReader); - public CHNativeBlock next() { - long block = nativeNext(nativeShuffleReader); - return new CHNativeBlock(block); - } + public CHNativeBlock next() { + long block = nativeNext(nativeShuffleReader); + return new CHNativeBlock(block); + } - private native void nativeClose(long shuffleReader); + private native void nativeClose(long shuffleReader); - @Override - public void close() throws Exception { - nativeClose(nativeShuffleReader); - nativeShuffleReader = 0L; - } + @Override + public void close() throws Exception { + nativeClose(nativeShuffleReader); + nativeShuffleReader = 0L; + } } diff --git a/backends-clickhouse/src/main/resources/META-INF/services/io.glutenproject.backendsapi.IIteratorApi b/backends-clickhouse/src/main/resources/META-INF/services/io.glutenproject.backendsapi.IIteratorApi new file mode 100644 index 000000000000..8771f0b70990 --- /dev/null +++ b/backends-clickhouse/src/main/resources/META-INF/services/io.glutenproject.backendsapi.IIteratorApi @@ -0,0 +1 @@ +io.glutenproject.backendsapi.clickhouse.CHIteratorApi \ No newline at end of file diff --git a/backends-clickhouse/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ISparkPlanExecApi b/backends-clickhouse/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ISparkPlanExecApi new file mode 100644 index 000000000000..93fcaae5fbdb --- /dev/null +++ b/backends-clickhouse/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ISparkPlanExecApi @@ -0,0 +1 @@ +io.glutenproject.backendsapi.clickhouse.CHSparkPlanExecApi \ No newline at end of file diff --git a/backends-clickhouse/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ITransformerApi b/backends-clickhouse/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ITransformerApi new file mode 100644 index 000000000000..f8d4912b1d99 --- /dev/null +++ b/backends-clickhouse/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ITransformerApi @@ -0,0 +1 @@ +io.glutenproject.backendsapi.clickhouse.CHTransformerApi \ No newline at end of file diff --git a/jvm/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/backends-clickhouse/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister similarity index 100% rename from jvm/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister rename to backends-clickhouse/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHIteratorApi.scala b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHIteratorApi.scala new file mode 100644 index 000000000000..22edb1e14d62 --- /dev/null +++ b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHIteratorApi.scala @@ -0,0 +1,286 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.backendsapi.clickhouse + +import scala.collection.JavaConverters._ + +import io.glutenproject.{GlutenConfig, GlutenNumaBindingInfo} +import io.glutenproject.backendsapi.IIteratorApi +import io.glutenproject.execution._ +import io.glutenproject.substrait.plan.PlanNode +import io.glutenproject.substrait.rel.{ExtensionTableBuilder, LocalFilesBuilder} +import io.glutenproject.vectorized.{ExpressionEvaluatorJniWrapper, _} +import org.apache.spark.{InterruptibleIterator, SparkConf, TaskContext} + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.FilePartition +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.vectorized.ColumnarBatch + +class CHIteratorApi extends IIteratorApi { + + /** + * Generate native row partition. + * + * @return + */ + override def genNativeFilePartition(p: InputPartition, + wsCxt: WholestageTransformContext + ): BaseNativeFilePartition = { + p match { + case p: NativeMergeTreePartition => + val extensionTableNode = + ExtensionTableBuilder.makeExtensionTable(p.minParts, + p.maxParts, p.database, p.table, p.tablePath) + wsCxt.substraitContext.setExtensionTableNode(extensionTableNode) + // logWarning(s"The substrait plan for partition " + + // s"${p.index}:\n${wsCxt.root.toProtobuf.toString}") + p.copySubstraitPlan(wsCxt.root.toProtobuf.toByteArray) + case FilePartition(index, files) => + val paths = new java.util.ArrayList[String]() + val starts = new java.util.ArrayList[java.lang.Long]() + val lengths = new java.util.ArrayList[java.lang.Long]() + files.foreach { f => + paths.add(f.filePath) + starts.add(new java.lang.Long(f.start)) + lengths.add(new java.lang.Long(f.length)) + } + val localFilesNode = LocalFilesBuilder.makeLocalFiles(index, paths, starts, lengths) + wsCxt.substraitContext.setLocalFilesNode(localFilesNode) + val substraitPlan = wsCxt.root.toProtobuf + /* + val out = new DataOutputStream(new FileOutputStream("/tmp/SubStraitTest-Q6.dat", + false)); + out.write(substraitPlan.toByteArray()); + out.flush(); + */ + // logWarning(s"The substrait plan for partition ${index}:\n${substraitPlan.toString}") + NativeFilePartition(index, files, substraitPlan.toByteArray) + } + } + + + /** + * Generate Iterator[ColumnarBatch] for CoalesceBatchesExec. + * + * @param iter + * @param recordsPerBatch + * @param numOutputRows + * @param numInputBatches + * @param numOutputBatches + * @param collectTime + * @param concatTime + * @param avgCoalescedNumRows + * @return + */ + override def genCoalesceIterator(iter: Iterator[ColumnarBatch], + recordsPerBatch: Int, + numOutputRows: SQLMetric, + numInputBatches: SQLMetric, + numOutputBatches: SQLMetric, + collectTime: SQLMetric, + concatTime: SQLMetric, + avgCoalescedNumRows: SQLMetric): Iterator[ColumnarBatch] = { + val operator = new CHCoalesceOperator(recordsPerBatch) + val res = new Iterator[ColumnarBatch] { + + override def hasNext: Boolean = { + val beforeNext = System.nanoTime + val hasNext = iter.hasNext + collectTime += System.nanoTime - beforeNext + hasNext + } + + override def next(): ColumnarBatch = { + val c = iter.next() + numInputBatches += 1 + val beforeConcat = System.nanoTime + operator.mergeBlock(c) + + while(!operator.isFull && iter.hasNext) { + val cb = iter.next(); + numInputBatches += 1; + operator.mergeBlock(cb) + } + val res = operator.release().toColumnarBatch + CHNativeBlock.fromColumnarBatch(res).ifPresent(block => { + numOutputRows += block.numRows(); + numOutputBatches += 1; + }) + res + } + + TaskContext.get().addTaskCompletionListener[Unit] { _ => + operator.close() + } + } + + new CloseableCHColumnBatchIterator(res) + } + + + /** + * Generate closeable ColumnBatch iterator. + * + * @param iter + * @return + */ + override def genCloseableColumnBatchIterator(iter: Iterator[ColumnarBatch] + ): Iterator[ColumnarBatch] = { + new CloseableCHColumnBatchIterator(iter) + } + + /** + * Generate Iterator[ColumnarBatch] for first stage. + * + * @return + */ + override def genFirstStageIterator(inputPartition: BaseNativeFilePartition, + loadNative: Boolean, + outputAttributes: Seq[Attribute], + context: TaskContext, + jarList: Seq[String]): Iterator[ColumnarBatch] = { + var resIter : AbstractBatchIterator = null + if (loadNative) { + // TODO: 'jarList' is kept for codegen + val transKernel = new ExpressionEvaluator(jarList.asJava) + val inBatchIters = new java.util.ArrayList[AbstractColumnarNativeIterator]() + resIter = transKernel.createKernelWithBatchIterator( + inputPartition.substraitPlan, inBatchIters) + TaskContext.get().addTaskCompletionListener[Unit] { _ => resIter.close() } + } + val iter = new Iterator[Any] { + // private val inputMetrics = TaskContext.get().taskMetrics().inputMetrics + + override def hasNext: Boolean = { + if (loadNative) { + resIter.hasNext + } else { + false + } + } + + override def next(): Any = { + if (!hasNext) { + throw new java.util.NoSuchElementException("End of stream") + } + resIter.next() + } + } + + // TODO: SPARK-25083 remove the type erasure hack in data source scan + new InterruptibleIterator(context, + new CloseableCHColumnBatchIterator(iter.asInstanceOf[Iterator[ColumnarBatch]])) + } + + /** + * Generate Iterator[ColumnarBatch] for final stage. + * + * @return + */ + override def genFinalStageIterator(iter: Iterator[ColumnarBatch], + numaBindingInfo: GlutenNumaBindingInfo, + listJars: Seq[String], + signature: String, + sparkConf: SparkConf, + outputAttributes: Seq[Attribute], + rootNode: PlanNode, + streamedSortPlan: SparkPlan, + pipelineTime: SQLMetric, + buildRelationBatchHolder: Seq[ColumnarBatch], + dependentKernels: Seq[ExpressionEvaluator], + dependentKernelIterators: Seq[AbstractBatchIterator] + ): Iterator[ColumnarBatch] = { + var build_elapse: Long = 0 + var eval_elapse: Long = 0 + GlutenConfig.getConf + val transKernel = new ExpressionEvaluator() + val inBatchIter = new ColumnarNativeIterator(iter.asJava) + val inBatchIters = new java.util.ArrayList[AbstractColumnarNativeIterator]() + inBatchIters.add(inBatchIter) + // we need to complete dependency RDD's firstly + val beforeBuild = System.nanoTime() + val nativeIterator = transKernel.createKernelWithBatchIterator(rootNode, inBatchIters) + build_elapse += System.nanoTime() - beforeBuild + val resIter = streamedSortPlan match { + case t: TransformSupport => + new Iterator[ColumnarBatch] { + override def hasNext: Boolean = { + val res = nativeIterator.hasNext + res + } + + override def next(): ColumnarBatch = { + val beforeEval = System.nanoTime() + nativeIterator.next() + } + } + case _ => + throw new UnsupportedOperationException( + s"streamedSortPlan should support transformation") + } + var closed = false + + def close = { + closed = true + pipelineTime += (eval_elapse + build_elapse) / 1000000 + buildRelationBatchHolder.foreach(_.close) // fixing: ref cnt goes nagative + dependentKernels.foreach(_.close) + dependentKernelIterators.foreach(_.close) + nativeIterator.close() + // relationHolder.clear() + } + + TaskContext.get().addTaskCompletionListener[Unit] { _ => + close + } + new CloseableCHColumnBatchIterator(resIter) + } + + /** + * Generate columnar native iterator. + * + * @return + */ + override def genColumnarNativeIterator(delegated: Iterator[ColumnarBatch] + ): ColumnarNativeIterator = { + new ColumnarNativeIterator(delegated.asJava) + } + + /** + * Generate BatchIterator for ExpressionEvaluator. + * + * @return + */ + override def genBatchIterator(wsPlan: Array[Byte], + iterList: Seq[AbstractColumnarNativeIterator], + jniWrapper: ExpressionEvaluatorJniWrapper + ): AbstractBatchIterator = { + val batchIteratorInstance = jniWrapper.nativeCreateKernelWithIterator( + 0L, wsPlan, iterList.toArray); + new BatchIterator(batchIteratorInstance) + } + + /** + * Get the backend api name. + * + * @return + */ + override def getBackendName: String = GlutenConfig.GLUTEN_CLICKHOUSE_BACKEND +} diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHSparkPlanExecApi.scala new file mode 100644 index 000000000000..5732fa3ab50a --- /dev/null +++ b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHSparkPlanExecApi.scala @@ -0,0 +1,111 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.backendsapi.clickhouse + +import io.glutenproject.backendsapi.ISparkPlanExecApi +import io.glutenproject.GlutenConfig +import io.glutenproject.execution.{NativeColumnarToRowExec, RowToArrowColumnarExec} +import io.glutenproject.vectorized.CHColumnarBatchSerializer +import org.apache.spark.ShuffleDependency + +import org.apache.spark.rdd.RDD +import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.{GenShuffleWriterParameters, GlutenShuffleWriterWrapper} +import org.apache.spark.shuffle.utils.CHShuffleUtil +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.utils.CHExecUtil +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch + +class CHSparkPlanExecApi extends ISparkPlanExecApi { + + /** + * Generate NativeColumnarToRowExec. + * + * @param child + * @return + */ + override def genNativeColumnarToRowExec(child: SparkPlan): NativeColumnarToRowExec = + throw new UnsupportedOperationException( + "Cannot support NativeColumnarToRowExec operation with ClickHouse backend.") + + + /** + * Generate RowToArrowColumnarExec. + * + * @param child + * @return + */ + override def genRowToArrowColumnarExec(child: SparkPlan): RowToArrowColumnarExec = + throw new UnsupportedOperationException( + "Cannot support RowToArrowColumnarExec operation with ClickHouse backend.") + + /** + * Generate ShuffleDependency for ColumnarShuffleExchangeExec. + * + * @return + */ + override def genShuffleDependency(rdd: RDD[ColumnarBatch], + outputAttributes: Seq[Attribute], + newPartitioning: Partitioning, + serializer: Serializer, + writeMetrics: Map[String, SQLMetric], + dataSize: SQLMetric, + bytesSpilled: SQLMetric, + numInputRows: SQLMetric, + computePidTime: SQLMetric, + splitTime: SQLMetric, + spillTime: SQLMetric, + compressTime: SQLMetric, + prepareTime: SQLMetric + ): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = { + CHExecUtil.genShuffleDependency(rdd, outputAttributes, newPartitioning, serializer, + writeMetrics, dataSize, bytesSpilled, numInputRows, computePidTime, splitTime, + spillTime, compressTime, prepareTime) + } + + /** + * Generate ColumnarShuffleWriter for ColumnarShuffleManager. + * + * @return + */ + override def genColumnarShuffleWriter[K, V](parameters: GenShuffleWriterParameters[K, V] + ): GlutenShuffleWriterWrapper[K, V] = { + CHShuffleUtil.genColumnarShuffleWriter(parameters) + } + + /** + * Generate ColumnarBatchSerializer for ColumnarShuffleExchangeExec. + * + * @return + */ + override def createColumnarBatchSerializer(schema: StructType, + readBatchNumRows: SQLMetric, + numOutputRows: SQLMetric): Serializer = { + new CHColumnarBatchSerializer(readBatchNumRows, numOutputRows) + } + + /** + * Get the backend api name. + * + * @return + */ + override def getBackendName: String = GlutenConfig.GLUTEN_CLICKHOUSE_BACKEND +} diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHTransformerApi.scala b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHTransformerApi.scala new file mode 100644 index 000000000000..fc7da853caf6 --- /dev/null +++ b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHTransformerApi.scala @@ -0,0 +1,42 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.backendsapi.clickhouse + +import io.glutenproject.backendsapi.ITransformerApi +import io.glutenproject.GlutenConfig + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.Partitioning + +class CHTransformerApi extends ITransformerApi { + + /** + * Do validate for ColumnarShuffleExchangeExec. + * For ClickHouse backend, it will return true directly. + * @return + */ + override def validateColumnarShuffleExchangeExec(outputPartitioning: Partitioning, + outputAttributes: Seq[Attribute] + ) = true + + /** + * Get the backend api name. + * + * @return + */ + override def getBackendName: String = GlutenConfig.GLUTEN_CLICKHOUSE_BACKEND +} diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/execution/NativeMergeTreePartition.scala b/backends-clickhouse/src/main/scala/io/glutenproject/execution/NativeMergeTreePartition.scala new file mode 100644 index 000000000000..d0b960e736fb --- /dev/null +++ b/backends-clickhouse/src/main/scala/io/glutenproject/execution/NativeMergeTreePartition.scala @@ -0,0 +1,34 @@ +/* + * 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 io.glutenproject.execution + +case class NativeMergeTreePartition(index: Int, engine: String, + database: String, + table: String, tablePath: String, + minParts: Long, maxParts: Long, + substraitPlan: Array[Byte] = Array.empty[Byte]) + extends BaseNativeFilePartition { + override def preferredLocations(): Array[String] = { + Array.empty[String] + } + + def copySubstraitPlan(newSubstraitPlan: Array[Byte]): NativeMergeTreePartition = { + this.copy(substraitPlan = newSubstraitPlan) + } +} + diff --git a/jvm/src/main/scala/io/glutenproject/vectorized/CHColumnarBatchSerializer.scala b/backends-clickhouse/src/main/scala/io/glutenproject/vectorized/CHColumnarBatchSerializer.scala similarity index 99% rename from jvm/src/main/scala/io/glutenproject/vectorized/CHColumnarBatchSerializer.scala rename to backends-clickhouse/src/main/scala/io/glutenproject/vectorized/CHColumnarBatchSerializer.scala index 66228b16fbb7..f92060749445 100644 --- a/jvm/src/main/scala/io/glutenproject/vectorized/CHColumnarBatchSerializer.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/vectorized/CHColumnarBatchSerializer.scala @@ -17,16 +17,18 @@ package io.glutenproject.vectorized +import java.io.{EOFException, InputStream, OutputStream, Serializable} +import java.nio.ByteBuffer + +import scala.reflect.ClassTag + import org.apache.spark.SparkEnv + import org.apache.spark.internal.Logging import org.apache.spark.serializer.{DeserializationStream, SerializationStream, Serializer, SerializerInstance} import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.vectorized.ColumnarBatch -import java.io.{EOFException, InputStream, OutputStream, Serializable} -import java.nio.ByteBuffer -import scala.reflect.ClassTag - class CHColumnarBatchSerializer(readBatchNumRows: SQLMetric, numOutputRows: SQLMetric) extends Serializer with Serializable { /** Creates a new [[SerializerInstance]]. */ diff --git a/jvm/src/main/scala/io/glutenproject/vectorized/CloseableCHColumnBatchIterator.scala b/backends-clickhouse/src/main/scala/io/glutenproject/vectorized/CloseableCHColumnBatchIterator.scala similarity index 86% rename from jvm/src/main/scala/io/glutenproject/vectorized/CloseableCHColumnBatchIterator.scala rename to backends-clickhouse/src/main/scala/io/glutenproject/vectorized/CloseableCHColumnBatchIterator.scala index 2ac000401902..abdf6e68c46f 100644 --- a/jvm/src/main/scala/io/glutenproject/vectorized/CloseableCHColumnBatchIterator.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/vectorized/CloseableCHColumnBatchIterator.scala @@ -18,17 +18,16 @@ package io.glutenproject.vectorized import org.apache.spark.TaskContext + import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils import org.apache.spark.sql.vectorized.ColumnarBatch /** * An Iterator that insures that the batches [[ColumnarBatch]]s it iterates over are all closed * properly. */ -class CloseableCHColumnBatchIterator(itr: Iterator[ColumnarBatch]) - extends Iterator[ColumnarBatch] - with Logging { +class CloseableCHColumnBatchIterator(itr: Iterator[ColumnarBatch]) extends + Iterator[ColumnarBatch] with Logging { var cb: ColumnarBatch = null private def closeCurrentBatch(): Unit = { @@ -43,9 +42,9 @@ class CloseableCHColumnBatchIterator(itr: Iterator[ColumnarBatch]) } } - SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit]((tc: TaskContext) => { - closeCurrentBatch() - }) + TaskContext.get().addTaskCompletionListener[Unit] { _ => + closeCurrentBatch() + } override def hasNext: Boolean = { itr.hasNext diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/CHColumnarShuffleWriter.scala b/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/CHColumnarShuffleWriter.scala new file mode 100644 index 000000000000..be48c1afda40 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/CHColumnarShuffleWriter.scala @@ -0,0 +1,192 @@ +/* + * 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.shuffle + +import java.io.IOException + +import scala.collection.mutable.ArrayBuffer + +import com.google.common.annotations.VisibleForTesting +import io.glutenproject.GlutenConfig +import io.glutenproject.vectorized._ +import org.apache.spark._ + +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.MapStatus +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.Utils + +class CHColumnarShuffleWriter[K, V]( + shuffleBlockResolver: IndexShuffleBlockResolver, + handle: BaseShuffleHandle[K, V, V], + mapId: Long, + writeMetrics: ShuffleWriteMetricsReporter) + extends ShuffleWriter[K, V] + with Logging { + + private val dep = handle.dependency.asInstanceOf[ColumnarShuffleDependency[K, V, V]] + + private val conf = SparkEnv.get.conf + + private val blockManager = SparkEnv.get.blockManager + + // Are we in the process of stopping? Because map tasks can call stop() with success = true + // and then call stop() with success = false if they get an exception, we want to make sure + // we don't try deleting files, etc twice. + private var stopping = false + + private var mapStatus: MapStatus = _ + + private val localDirs = blockManager.diskBlockManager.localDirs.mkString(",") + + private val offheapSize = conf.getSizeAsBytes("spark.memory.offHeap.size", 0) + private val executorNum = conf.getInt("spark.executor.cores", 1) + private val offheapPerTask = offheapSize / executorNum; + + private val nativeBufferSize = GlutenConfig.getConf.shuffleSplitDefaultSize + + private val customizedCompressCodec = + GlutenConfig.getConf.columnarShuffleUseCustomizedCompressionCodec + private val defaultCompressionCodec = if (conf.getBoolean("spark.shuffle.compress", true)) { + conf.get("spark.io.compression.codec", "lz4") + } else { + "uncompressed" + } + private val batchCompressThreshold = + GlutenConfig.getConf.columnarShuffleBatchCompressThreshold; + + private val preferSpill = GlutenConfig.getConf.columnarShufflePreferSpill + + private val writeSchema = GlutenConfig.getConf.columnarShuffleWriteSchema + + private val jniWrapper = new CHShuffleSplitterJniWrapper + + private var nativeSplitter: Long = 0 + + private var splitResult: SplitResult = _ + + private var partitionLengths: Array[Long] = _ + + private var rawPartitionLengths: Array[Long] = _ + + private var firstRecordBatch: Boolean = true + + def internalCHWrite(records: Iterator[Product2[K, V]]): Unit = { + val splitterJniWrapper : CHShuffleSplitterJniWrapper = + jniWrapper.asInstanceOf[CHShuffleSplitterJniWrapper] + if (!records.hasNext) { + partitionLengths = new Array[Long](dep.partitioner.numPartitions) + shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, null) + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapId) + return + } + val dataTmp = Utils.tempFileWith(shuffleBlockResolver.getDataFile(dep.shuffleId, mapId)) + if (nativeSplitter == 0) { + nativeSplitter = splitterJniWrapper.make( + dep.nativePartitioning, + mapId, + nativeBufferSize, + defaultCompressionCodec, + dataTmp.getAbsolutePath, + localDirs) + } + while (records.hasNext) { + val cb = records.next()._2.asInstanceOf[ColumnarBatch] + if (cb.numRows == 0 || cb.numCols == 0) { + logInfo(s"Skip ColumnarBatch of ${cb.numRows} rows, ${cb.numCols} cols") + } else { + val startTimeForPrepare = System.nanoTime() + + val startTime = System.nanoTime() + firstRecordBatch = false + dep.prepareTime.add(System.nanoTime() - startTimeForPrepare) + val col = cb.column(0).asInstanceOf[CHColumnVector] + val block = col.getBlockAddress + splitterJniWrapper + .split(nativeSplitter, cb.numRows, block) + dep.splitTime.add(System.nanoTime() - startTime) + dep.numInputRows.add(cb.numRows) + writeMetrics.incRecordsWritten(1) + } + } + val startTime = System.nanoTime() + splitResult = splitterJniWrapper.stop(nativeSplitter) + + dep.splitTime.add(System.nanoTime() - startTime - splitResult.getTotalSpillTime - + splitResult.getTotalWriteTime - splitResult.getTotalComputePidTime - + splitResult.getTotalCompressTime) + dep.spillTime.add(splitResult.getTotalSpillTime) + dep.compressTime.add(splitResult.getTotalCompressTime) + dep.computePidTime.add(splitResult.getTotalComputePidTime) + dep.bytesSpilled.add(splitResult.getTotalBytesSpilled) + writeMetrics.incBytesWritten(splitResult.getTotalBytesWritten) + writeMetrics.incWriteTime(splitResult.getTotalWriteTime + splitResult.getTotalSpillTime) + + partitionLengths = splitResult.getPartitionLengths + rawPartitionLengths = splitResult.getRawPartitionLengths + try { + shuffleBlockResolver.writeIndexFileAndCommit( + dep.shuffleId, + mapId, + partitionLengths, + dataTmp) + } finally { + if (dataTmp.exists() && !dataTmp.delete()) { + logError(s"Error while deleting temp file ${dataTmp.getAbsolutePath}") + } + } + + // fixme workaround: to store uncompressed sizes on the rhs of (maybe) compressed sizes + val unionPartitionLengths = ArrayBuffer[Long]() + unionPartitionLengths ++= partitionLengths + unionPartitionLengths ++= rawPartitionLengths + mapStatus = MapStatus(blockManager.shuffleServerId, unionPartitionLengths.toArray, mapId) + } + + @throws[IOException] + override def write(records: Iterator[Product2[K, V]]): Unit = { + internalCHWrite(records) + } + + def closeCHSplitter(): Unit = { + jniWrapper.asInstanceOf[CHShuffleSplitterJniWrapper].close(nativeSplitter) + } + + override def stop(success: Boolean): Option[MapStatus] = { + try { + if (stopping) { + None + } + stopping = true + if (success) { + Option(mapStatus) + } else { + None + } + } finally { + if (nativeSplitter != 0) { + closeCHSplitter() + nativeSplitter = 0 + } + } + } + + @VisibleForTesting + def getPartitionLengths: Array[Long] = partitionLengths + +} diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/utils/CHShuffleUtil.scala b/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/utils/CHShuffleUtil.scala new file mode 100644 index 000000000000..76d0cebc8b64 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/utils/CHShuffleUtil.scala @@ -0,0 +1,31 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle.utils + +import org.apache.spark.shuffle.{CHColumnarShuffleWriter, GenShuffleWriterParameters, GlutenShuffleWriterWrapper} + +object CHShuffleUtil { + + def genColumnarShuffleWriter[K, V](parameters: GenShuffleWriterParameters[K, V] + ): GlutenShuffleWriterWrapper[K, V] = { + GlutenShuffleWriterWrapper(new CHColumnarShuffleWriter[K, V]( + parameters.shuffleBlockResolver, + parameters.columnarShuffleHandle, + parameters.mapId, + parameters.metrics)) + } +} diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala similarity index 100% rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala rename to backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseDataSource.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseDataSource.scala similarity index 98% rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseDataSource.scala rename to backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseDataSource.scala index 57dbc6708729..cb0ed2e4bd9b 100644 --- a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseDataSource.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseDataSource.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.datasources.v2.clickhouse -import scala.collection.JavaConverters._ - import org.apache.spark.sql.connector.catalog.TableProvider import org.apache.spark.sql.execution.datasources.v2.clickhouse.table.ClickHouseTableV2 diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseLog.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseLog.scala similarity index 100% rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseLog.scala rename to backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseLog.scala diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala similarity index 99% rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala rename to backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala index e44b93303b80..1d885100c729 100644 --- a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.datasources.v2.clickhouse -import scala.collection.JavaConverters._ import scala.collection.mutable import java.util diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/commands/CreateClickHouseTableCommand.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/commands/CreateClickHouseTableCommand.scala similarity index 100% rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/commands/CreateClickHouseTableCommand.scala rename to backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/commands/CreateClickHouseTableCommand.scala diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala similarity index 100% rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala rename to backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/ClickHousePartitionReaderFactory.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/ClickHousePartitionReaderFactory.scala similarity index 100% rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/ClickHousePartitionReaderFactory.scala rename to backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/ClickHousePartitionReaderFactory.scala diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/ClickHouseScan.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/ClickHouseScan.scala similarity index 100% rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/ClickHouseScan.scala rename to backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/ClickHouseScan.scala diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/ClickHouseScanBuilder.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/ClickHouseScanBuilder.scala similarity index 100% rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/ClickHouseScanBuilder.scala rename to backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/ClickHouseScanBuilder.scala diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/table/ClickHouseTableV2.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/table/ClickHouseTableV2.scala similarity index 100% rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/table/ClickHouseTableV2.scala rename to backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/table/ClickHouseTableV2.scala diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/utils/CHDataSourceUtils.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/utils/CHDataSourceUtils.scala similarity index 100% rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/utils/CHDataSourceUtils.scala rename to backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/utils/CHDataSourceUtils.scala diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/utils/ScanMergeTreePartsUtils.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/utils/ScanMergeTreePartsUtils.scala similarity index 100% rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/utils/ScanMergeTreePartsUtils.scala rename to backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/utils/ScanMergeTreePartsUtils.scala diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/utils/CHExecUtil.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/utils/CHExecUtil.scala new file mode 100644 index 000000000000..fd0cf54436ea --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/utils/CHExecUtil.scala @@ -0,0 +1,106 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.utils + +import scala.collection.JavaConverters._ + +import io.glutenproject.expression.ConverterUtils +import io.glutenproject.vectorized.NativePartitioning +import org.apache.spark.ShuffleDependency + +import org.apache.spark.rdd.RDD +import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.ColumnarShuffleDependency +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.PartitionIdPassthrough +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.vectorized.ColumnarBatch + +object CHExecUtil { + + def genShuffleDependency(rdd: RDD[ColumnarBatch], + outputAttributes: Seq[Attribute], + newPartitioning: Partitioning, + serializer: Serializer, + writeMetrics: Map[String, SQLMetric], + dataSize: SQLMetric, + bytesSpilled: SQLMetric, + numInputRows: SQLMetric, + computePidTime: SQLMetric, + splitTime: SQLMetric, + spillTime: SQLMetric, + compressTime: SQLMetric, + prepareTime: SQLMetric + ): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = { + val nativePartitioning: NativePartitioning = newPartitioning match { + case SinglePartition => new NativePartitioning("single", 1, Array.empty[Byte]) + case RoundRobinPartitioning(n) => + new NativePartitioning("rr", n, Array.empty[Byte]) + case HashPartitioning(exprs, n) => + val fields = exprs.zipWithIndex.map { + case (expr, i) => + val attr = ConverterUtils.getAttrFromExpr(expr) + attr.name + } + new NativePartitioning( + "hash", + n, + null, + fields.mkString(",").getBytes) + } + + val isRoundRobin = newPartitioning.isInstanceOf[RoundRobinPartitioning] && + newPartitioning.numPartitions > 1 + + // RDD passed to ShuffleDependency should be the form of key-value pairs. + // ColumnarShuffleWriter will compute ids from ColumnarBatch on native side other than read the "key" part. + // Thus in Columnar Shuffle we never use the "key" part. + val isOrderSensitive = isRoundRobin && !SQLConf.get.sortBeforeRepartition + + val rddWithDummyKey: RDD[Product2[Int, ColumnarBatch]] = newPartitioning match { + case _ => + rdd.mapPartitionsWithIndexInternal( + (_, cbIter) => + cbIter.map { cb => + (0, cb) + }, + isOrderSensitive = isOrderSensitive) + } + + val dependency = + new ColumnarShuffleDependency[Int, ColumnarBatch, ColumnarBatch]( + rddWithDummyKey, + new PartitionIdPassthrough(newPartitioning.numPartitions), + serializer, + shuffleWriterProcessor = + ShuffleExchangeExec.createShuffleWriteProcessor(writeMetrics), + nativePartitioning = nativePartitioning, + dataSize = dataSize, + bytesSpilled = bytesSpilled, + numInputRows = numInputRows, + computePidTime = computePidTime, + splitTime = splitTime, + spillTime = spillTime, + compressTime = compressTime, + prepareTime = prepareTime) + + dependency + } +} diff --git a/jvm/src/test/scala/io/glutenproject/jni/TestExpressionEvaluator.java b/backends-clickhouse/src/test/java/io/glutenproject/jni/TestExpressionEvaluator.java similarity index 77% rename from jvm/src/test/scala/io/glutenproject/jni/TestExpressionEvaluator.java rename to backends-clickhouse/src/test/java/io/glutenproject/jni/TestExpressionEvaluator.java index e6d5bb75b859..4a1d5948bf99 100644 --- a/jvm/src/test/scala/io/glutenproject/jni/TestExpressionEvaluator.java +++ b/backends-clickhouse/src/test/java/io/glutenproject/jni/TestExpressionEvaluator.java @@ -1,19 +1,16 @@ package io.glutenproject.jni; -import io.glutenproject.execution.ColumnarNativeIterator; +import io.glutenproject.execution.AbstractColumnarNativeIterator; import io.glutenproject.row.RowIterator; import io.glutenproject.row.SparkRowInfo; import io.glutenproject.vectorized.ExpressionEvaluator; import io.substrait.proto.Plan; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; import java.io.DataInputStream; import java.io.FileInputStream; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; -import java.util.List; public class TestExpressionEvaluator { @@ -25,7 +22,7 @@ public void setup() { public void testInitNative() throws Exception { ExpressionEvaluator transKernel = new ExpressionEvaluator(new ArrayList(), "spark_columnar_jni", - "/home/myubuntu/Works/c_cpp_projects/Kyligence-ClickHouse-MergeTree/cmake-build-debug/utils/local-engine/liblocal_engine_jnid.so", + "/usr/local/clickhouse/lib/libch.so", null, false); transKernel.initNative(); @@ -40,7 +37,7 @@ public void testInitNative() throws Exception { Plan substraitPlan = Plan.parseFrom(dis); - ArrayList inBatchIters = new java.util.ArrayList(); + ArrayList inBatchIters = new ArrayList(); RowIterator resIter = transKernel.createKernelWithRowIterator(substraitPlan.toByteArray(), inBatchIters); while (resIter.hasNext()) { diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2BenchmarkTest.scala b/backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2BenchmarkTest.scala new file mode 100644 index 000000000000..e807ff6305f4 --- /dev/null +++ b/backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2BenchmarkTest.scala @@ -0,0 +1,645 @@ +/* + * 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 io.glutenproject.benchmarks + +import java.io.File + +import scala.collection.mutable.ArrayBuffer +import scala.io.Source + +import io.glutenproject.GlutenConfig + +import org.apache.spark.sql.SparkSession + +object DSV2BenchmarkTest { + + val tableName = "lineitem_ch" + + def main(args: Array[String]): Unit = { + + val (parquetFilesPath, fileFormat, + executedCnt, configed, sqlFilePath, stopFlagFile, + createTable, metaRootPath) = if (args.length > 0) { + (args(0), args(1), args(2).toInt, true, args(3), args(4), args(5).toBoolean, args(6)) + } else { + val rootPath = this.getClass.getResource("/").getPath + val resourcePath = rootPath + "../../../../jvm/src/test/resources/" + val dataPath = resourcePath + "/tpch-data/" + val queryPath = resourcePath + "/queries/" + //(new File(dataPath).getAbsolutePath, "parquet", 1, false, queryPath + "q06.sql", "", true, + //"/data1/gazelle-jni-warehouse") + ("/data1/test_output/tpch-data-sf10", "parquet", 1, false, queryPath + "q01.sql", "", + true, "/data1/gazelle-jni-warehouse") + } + + val (warehouse, metaStorePathAbsolute, hiveMetaStoreDB) = if (!metaRootPath.isEmpty) { + (metaRootPath + "/spark-warehouse", metaRootPath + "/meta", + metaRootPath + "/meta/metastore_db") + } else { + ("/tmp/spark-warehouse", "/tmp/meta", "/tmp/meta/metastore_db") + } + + if (!warehouse.isEmpty) { + val warehouseDir = new File(warehouse) + if (!warehouseDir.exists()) { + warehouseDir.mkdirs() + } + val hiveMetaStoreDBDir = new File(metaStorePathAbsolute) + if (!hiveMetaStoreDBDir.exists()) { + hiveMetaStoreDBDir.mkdirs() + } + } + + val sqlStr = Source.fromFile(new File(sqlFilePath), "UTF-8") + + val sessionBuilderTmp = SparkSession + .builder() + .appName("Gluten-Benchmark") + + val sessionBuilder = if (!configed) { + val sessionBuilderTmp1 = sessionBuilderTmp + .master("local[3]") + .config("spark.driver.memory", "4G") + .config("spark.driver.memoryOverhead", "6G") + .config("spark.serializer", "org.apache.spark.serializer.JavaSerializer") + .config("spark.default.parallelism", 1) + .config("spark.sql.shuffle.partitions", 1) + .config("spark.sql.adaptive.enabled", "false") + .config("spark.sql.files.maxPartitionBytes", 1024 << 10 << 10) // default is 128M + .config("spark.sql.files.minPartitionNum", "1") + .config("spark.sql.parquet.filterPushdown", "true") + .config("spark.locality.wait", "0s") + .config("spark.sql.sources.ignoreDataLocality", "true") + .config("spark.sql.parquet.enableVectorizedReader", "true") + //.config("spark.sql.sources.useV1SourceList", "avro") + .config("spark.memory.fraction", "0.3") + .config("spark.memory.storageFraction", "0.3") + //.config("spark.sql.parquet.columnarReaderBatchSize", "20000") + .config("spark.plugins", "io.glutenproject.GlutenPlugin") + .config("spark.sql.catalog.spark_catalog", + "org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseSparkCatalog") + .config("spark.databricks.delta.maxSnapshotLineageLength", 20) + .config("spark.databricks.delta.snapshotPartitions", 1) + .config("spark.databricks.delta.properties.defaults.checkpointInterval", 5) + .config("spark.databricks.delta.stalenessLimit", 3600 * 1000) + .config("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + //.config("spark.sql.execution.arrow.maxRecordsPerBatch", "20000") + .config("spark.gluten.sql.columnar.columnartorow", "false") + .config("spark.gluten.sql.columnar.backend.lib", "ch") + .config(GlutenConfig.GLUTEN_LOAD_NATIVE, "true") + .config(GlutenConfig.GLUTEN_LOAD_ARROW, "false") + .config(GlutenConfig.GLUTEN_LIB_PATH, + "/usr/local/clickhouse/lib/libch.so") + .config("spark.gluten.sql.columnar.iterator", "true") + .config("spark.gluten.sql.columnar.hashagg.enablefinal", "true") + .config("spark.gluten.sql.enable.native.validation", "false") + //.config("spark.sql.planChangeLog.level", "info") + .config("spark.sql.columnVector.offheap.enabled", "true") + .config("spark.memory.offHeap.enabled", "true") + .config("spark.memory.offHeap.size", "6442450944") + .config("spark.io.compression.codec", "LZ4") + + if (!warehouse.isEmpty) { + sessionBuilderTmp1.config("spark.sql.warehouse.dir", warehouse) + .config("javax.jdo.option.ConnectionURL", s"jdbc:derby:;databaseName=$hiveMetaStoreDB;create=true") + .enableHiveSupport() + } else { + sessionBuilderTmp1.enableHiveSupport() + } + } else { + sessionBuilderTmp + } + + val spark = sessionBuilder.getOrCreate() + if (!configed) { + spark.sparkContext.setLogLevel("WARN") + } + + val createTbl = false + if (createTbl) { + createClickHouseTables(spark, parquetFilesPath, fileFormat) + createLocationClickHouseTable(spark) + // createTables(spark, parquetFilesPath, fileFormat) + } + val refreshTable = false + if (refreshTable) { + refreshClickHouseTable(spark) + } + selectClickHouseTable(spark, executedCnt, sqlStr.mkString) + //selectLocationClickHouseTable(spark, executedCnt, sqlStr.mkString) + //testSerializeFromObjectExec(spark) + + selectQ1ClickHouseTable(spark, executedCnt, sqlStr.mkString) + //selectQ1LocationClickHouseTable(spark, executedCnt, sqlStr.mkString) + + //testSparkTPCH(spark) + //testSQL(spark, parquetFilesPath, fileFormat, executedCnt, sqlStr.mkString) + + System.out.println("waiting for finishing") + if (stopFlagFile.isEmpty) { + Thread.sleep(1800000) + } else { + while ((new File(stopFlagFile)).exists()) { + Thread.sleep(1000) + } + } + spark.stop() + System.out.println("finished") + } + + def testSerializeFromObjectExec(spark: SparkSession): Unit = { + //spark.conf.set("spark.gluten.sql.enable.native.engine", "false") + val tookTimeArr = Array(12, 23, 56, 100, 500, 20) + import spark.implicits._ + val df = spark.sparkContext.parallelize(tookTimeArr.toSeq, 1).toDF("time") + df.summary().show(100, false) + } + + def createClickHouseTables(spark: SparkSession, + parquetFilesPath: String, fileFormat: String): Unit = { + spark.sql( + """ + | show databases + |""".stripMargin).show(100, false) + + spark.sql( + """ + | show tables + |""".stripMargin).show(100, false) + + spark.sql( + s""" + | USE default + |""".stripMargin).show(100, false) + + // Clear up old session + spark.sql(s"DROP TABLE IF EXISTS $tableName") + + // Create a table + println("Creating a table") + // PARTITIONED BY (age) + // engine='MergeTree' or engine='Parquet' + spark.sql( + s""" + | CREATE TABLE IF NOT EXISTS $tableName ( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string) + | USING clickhouse + | TBLPROPERTIES (engine='MergeTree' + | ) + |""".stripMargin) + + spark.sql( + """ + | show tables + |""".stripMargin).show(100, false) + + spark.sql( + s""" + | desc formatted ${tableName} + |""".stripMargin).show(100, false) + + } + + def createLocationClickHouseTable(spark: SparkSession): Unit = { + spark.sql( + s""" + | USE default + |""".stripMargin).show(100, false) + + spark.sql( + """ + | show tables + |""".stripMargin).show(100, false) + + // Clear up old session + spark.sql(s"DROP TABLE IF EXISTS ch_clickhouse") + + // Create a table + // PARTITIONED BY (age) + // engine='MergeTree' or engine='Parquet' + spark.sql( + s""" + | CREATE TABLE IF NOT EXISTS ch_clickhouse ( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string) + | USING clickhouse + | TBLPROPERTIES (engine='MergeTree' + | ) + | LOCATION '/data1/gazelle-jni-warehouse/ch_clickhouse' + |""".stripMargin) + + spark.sql( + """ + | show tables + |""".stripMargin).show(100, false) + } + + def refreshClickHouseTable(spark: SparkSession): Unit = { + spark.sql( + s""" + | refresh table ${tableName} + |""".stripMargin).show(100, false) + spark.sql( + s""" + | desc formatted ${tableName} + |""".stripMargin).show(100, false) + spark.sql( + s""" + | refresh table ch_clickhouse + |""".stripMargin).show(100, false) + spark.sql( + s""" + | desc formatted ch_clickhouse + |""".stripMargin).show(100, false) + } + + def selectClickHouseTable(spark: SparkSession, executedCnt: Int, + sql: String): Unit = { + val tookTimeArr = ArrayBuffer[Long]() + for (i <- 1 to executedCnt) { + val startTime = System.nanoTime() + spark.sql( + s""" + |SELECT + | sum(l_extendedprice * l_discount) AS revenue + |FROM + | ${tableName} + |WHERE + | l_shipdate >= date'1994-01-01' + | AND l_shipdate < date'1994-01-01' + interval 1 year + | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 + | AND l_quantity < 24; + |""".stripMargin).show(200, false) + val tookTime = (System.nanoTime() - startTime) / 1000000 + println(s"Execute ${i} time, time: ${tookTime}") + tookTimeArr += tookTime + } + + println(tookTimeArr.mkString(",")) + + // spark.conf.set("spark.gluten.sql.enable.native.engine", "false") + import spark.implicits._ + val df = spark.sparkContext.parallelize(tookTimeArr.toSeq, 1).toDF("time") + df.summary().show(100, false) + } + + def selectLocationClickHouseTable(spark: SparkSession, executedCnt: Int, + sql: String): Unit = { + val tookTimeArr = ArrayBuffer[Long]() + for (i <- 1 to executedCnt) { + val startTime = System.nanoTime() + spark.sql( + s""" + |SELECT + | sum(l_extendedprice * l_discount) AS revenue + |FROM + | ch_clickhouse + |WHERE + | l_shipdate >= date'1994-01-01' + | AND l_shipdate < date'1994-01-01' + interval 1 year + | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 + | AND l_quantity < 24; + |""".stripMargin).show(200, false) + val tookTime = (System.nanoTime() - startTime) / 1000000 + println(s"Execute ${i} time, time: ${tookTime}") + tookTimeArr += tookTime + } + + println(tookTimeArr.mkString(",")) + + //spark.conf.set("spark.gluten.sql.enable.native.engine", "false") + import spark.implicits._ + val df = spark.sparkContext.parallelize(tookTimeArr.toSeq, 1).toDF("time") + df.summary().show(100, false) + } + + def selectQ1ClickHouseTable(spark: SparkSession, executedCnt: Int, + sql: String): Unit = { + val tookTimeArr = ArrayBuffer[Long]() + for (i <- 1 to executedCnt) { + val startTime = System.nanoTime() + spark.sql( + s""" + |SELECT + | l_returnflag, + | l_linestatus, + | sum(l_quantity) AS sum_qty, + | sum(l_extendedprice) AS sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, + | avg(l_quantity) AS avg_qty, + | avg(l_extendedprice) AS avg_price, + | avg(l_discount) AS avg_disc, + | count(*) AS count_order + |FROM + | ${tableName} + |WHERE + | l_shipdate <= date'1998-09-02' - interval 1 day + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; + |""".stripMargin).show(200, false) + // can not use .collect(), will lead to error. + val tookTime = (System.nanoTime() - startTime) / 1000000 + println(s"Execute ${i} time, time: ${tookTime}") + tookTimeArr += tookTime + } + + println(tookTimeArr.mkString(",")) + + // spark.conf.set("spark.gluten.sql.enable.native.engine", "false") + import spark.implicits._ + val df = spark.sparkContext.parallelize(tookTimeArr.toSeq, 1).toDF("time") + df.summary().show(100, false) + } + + def selectQ1LocationClickHouseTable(spark: SparkSession, executedCnt: Int, + sql: String): Unit = { + val tookTimeArr = ArrayBuffer[Long]() + for (i <- 1 to executedCnt) { + val startTime = System.nanoTime() + spark.sql( + s""" + |SELECT + | l_returnflag, + | l_linestatus, + | sum(l_quantity) AS sum_qty, + | sum(l_extendedprice) AS sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, + | avg(l_quantity) AS avg_qty, + | avg(l_extendedprice) AS avg_price, + | avg(l_discount) AS avg_disc, + | count(*) AS count_order + |FROM + | ch_clickhouse + |WHERE + | l_shipdate <= date'1998-09-02' - interval 1 day + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; + |""".stripMargin).show(200, false) + val tookTime = (System.nanoTime() - startTime) / 1000000 + println(s"Execute ${i} time, time: ${tookTime}") + tookTimeArr += tookTime + } + + println(tookTimeArr.mkString(",")) + + //spark.conf.set("spark.gluten.sql.enable.native.engine", "false") + import spark.implicits._ + val df = spark.sparkContext.parallelize(tookTimeArr.toSeq, 1).toDF("time") + df.summary().show(100, false) + } + + def testSQL(spark: SparkSession, parquetFilesPath: String, + fileFormat: String, executedCnt: Int, + sql: String): Unit = { + /*spark.sql( + s""" + | show tables + |""".stripMargin).show(100, false)*/ + + val tookTimeArr = ArrayBuffer[Long]() + for (i <- 1 to executedCnt) { + val startTime = System.nanoTime() + spark.sql(sql).show(200, false) + val tookTime = (System.nanoTime() - startTime) / 1000000 + println(s"Execute ${i} time, time: ${tookTime}") + tookTimeArr += tookTime + } + + println(tookTimeArr.mkString(",")) + + //spark.conf.set("spark.gluten.sql.enable.native.engine", "false") + import spark.implicits._ + val df = spark.sparkContext.parallelize(tookTimeArr.toSeq, 1).toDF("time") + df.summary().show(100, false) + } + + def testSparkTPCH(spark: SparkSession): Unit = { + val tookTimeArr = ArrayBuffer[Long]() + val rootPath = this.getClass.getResource("/").getPath + val resourcePath = rootPath + "../../../src/test/resources/" + val queryPath = resourcePath + "/queries/" + for (i <- 1 to 22) { + val startTime = System.nanoTime() + val sqlFile = queryPath + "q" + "%02d".format(i) + ".sql" + println(sqlFile) + val sqlStr = Source.fromFile(new File(sqlFile), "UTF-8") + spark.sql(sqlStr.mkString).collect() + val tookTime = (System.nanoTime() - startTime) / 1000000 + println(s"Execute ${i} time, time: ${tookTime}") + tookTimeArr += tookTime + } + println(tookTimeArr.mkString(",")) + } + + def createTables(spark: SparkSession, parquetFilesPath: String, fileFormat: String): Unit = { + val dataSourceMap = Map( + "customer" -> spark.read.format(fileFormat).load(parquetFilesPath + "/customer"), + + "lineitem" -> spark.read.format(fileFormat).load(parquetFilesPath + "/lineitem"), + + "nation" -> spark.read.format(fileFormat).load(parquetFilesPath + "/nation"), + + "region" -> spark.read.format(fileFormat).load(parquetFilesPath + "/region"), + + "orders" -> spark.read.format(fileFormat).load(parquetFilesPath + "/order"), + + "part" -> spark.read.format(fileFormat).load(parquetFilesPath + "/part"), + + "partsupp" -> spark.read.format(fileFormat).load(parquetFilesPath + "/partsupp"), + + "supplier" -> spark.read.format(fileFormat).load(parquetFilesPath + "/supplier")) + + val customerData = parquetFilesPath + "/customer" + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS customer ( + | c_custkey bigint, + | c_name string, + | c_address string, + | c_nationkey bigint, + | c_phone string, + | c_acctbal double, + | c_mktsegment string, + | c_comment string) + | STORED AS PARQUET LOCATION '${customerData}' + |""".stripMargin).show(1, false) + + val lineitemData = parquetFilesPath + "/lineitem" + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS lineitem ( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string) + | STORED AS PARQUET LOCATION '${lineitemData}' + |""".stripMargin).show(1, false) + + val nationData = parquetFilesPath + "/nation" + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS nation ( + | n_nationkey bigint, + | n_name string, + | n_regionkey bigint, + | n_comment string) + | STORED AS PARQUET LOCATION '${nationData}' + |""".stripMargin).show(1, false) + + val regionData = parquetFilesPath + "/region" + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS region ( + | r_regionkey bigint, + | r_name string, + | r_comment string) + | STORED AS PARQUET LOCATION '${regionData}' + |""".stripMargin).show(1, false) + + val ordersData = parquetFilesPath + "/order" + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS orders ( + | o_orderkey bigint, + | o_custkey bigint, + | o_orderstatus string, + | o_totalprice double, + | o_orderdate date, + | o_orderpriority string, + | o_clerk string, + | o_shippriority bigint, + | o_comment string) + | STORED AS PARQUET LOCATION '${ordersData}' + |""".stripMargin).show(1, false) + + val partData = parquetFilesPath + "/part" + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS part ( + | p_partkey bigint, + | p_name string, + | p_mfgr string, + | p_brand string, + | p_type string, + | p_size bigint, + | p_container string, + | p_retailprice double, + | p_comment string) + | STORED AS PARQUET LOCATION '${partData}' + |""".stripMargin).show(1, false) + + val partsuppData = parquetFilesPath + "/partsupp" + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS partsupp ( + | ps_partkey bigint, + | ps_suppkey bigint, + | ps_availqty bigint, + | ps_supplycost double, + | ps_comment string) + | STORED AS PARQUET LOCATION '${partsuppData}' + |""".stripMargin).show(1, false) + + val supplierData = parquetFilesPath + "/supplier" + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS supplier ( + | s_suppkey bigint, + | s_name string, + | s_address string, + | s_nationkey bigint, + | s_phone string, + | s_acctbal double, + | s_comment string) + | STORED AS PARQUET LOCATION '${supplierData}' + |""".stripMargin).show(1, false) + + /*spark.sql( + s""" + | show databases; + |""".stripMargin).show(100, false) + spark.sql( + s""" + | show tables; + |""".stripMargin).show(100, false) + dataSourceMap.foreach { + case (key, value) => { + println(s"----------------create table $key") + spark.sql( + s""" + | desc $key; + |""".stripMargin).show(100, false) + spark.sql( + s""" + | select count(1) from $key; + |""".stripMargin).show(10, false) + } + }*/ + } +} diff --git a/jvm/src/test/scala/io/glutenproject/benchmarks/DSV2ColumnarBenchmarkTest.scala b/backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2ColumnarBenchmarkTest.scala similarity index 99% rename from jvm/src/test/scala/io/glutenproject/benchmarks/DSV2ColumnarBenchmarkTest.scala rename to backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2ColumnarBenchmarkTest.scala index 7d5e1bf78c5e..0a28cfef6761 100644 --- a/jvm/src/test/scala/io/glutenproject/benchmarks/DSV2ColumnarBenchmarkTest.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2ColumnarBenchmarkTest.scala @@ -17,13 +17,15 @@ package io.glutenproject.benchmarks -import io.glutenproject.GlutenConfig -import org.apache.spark.sql.SparkSession - import java.io.File + import scala.collection.mutable.ArrayBuffer import scala.io.Source +import io.glutenproject.GlutenConfig + +import org.apache.spark.sql.SparkSession + object DSV2ColumnarBenchmarkTest { val tableName = "lineitem_ch" @@ -99,7 +101,7 @@ object DSV2ColumnarBenchmarkTest { .config(GlutenConfig.GLUTEN_LOAD_NATIVE, "true") .config(GlutenConfig.GLUTEN_LOAD_ARROW, "false") .config(GlutenConfig.GLUTEN_LIB_PATH, - "/home/saber/Documents/github/ClickHouse/cmake-build-relwithdebinfo/utils/local-engine/libch.so") + "/usr/local/clickhouse/lib/libch.so") .config("spark.gluten.sql.columnar.iterator", "true") //.config("spark.sql.planChangeLog.level", "info") .config("spark.sql.columnVector.offheap.enabled", "true") diff --git a/jvm/src/test/scala/io/glutenproject/benchmarks/DSV2BenchmarkTest.scala b/backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2Q1ColumnarBenchmarkTest.scala similarity index 82% rename from jvm/src/test/scala/io/glutenproject/benchmarks/DSV2BenchmarkTest.scala rename to backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2Q1ColumnarBenchmarkTest.scala index 6d65ecac6b7b..a3af9e950189 100644 --- a/jvm/src/test/scala/io/glutenproject/benchmarks/DSV2BenchmarkTest.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2Q1ColumnarBenchmarkTest.scala @@ -26,7 +26,7 @@ import io.glutenproject.GlutenConfig import org.apache.spark.sql.SparkSession -object DSV2BenchmarkTest { +object DSV2Q1ColumnarBenchmarkTest { val tableName = "lineitem_ch" @@ -41,8 +41,8 @@ object DSV2BenchmarkTest { val resourcePath = rootPath + "../../../src/test/resources/" val dataPath = resourcePath + "/tpch-data/" val queryPath = resourcePath + "/queries/" - (new File(dataPath).getAbsolutePath, "parquet", 1, false, queryPath + "q06.sql", "", true, - "/data1/gluten-warehouse") + (new File(dataPath).getAbsolutePath, "parquet", 1, false, queryPath + "q01.sql", "", true, + "/tmp/gluten-warehouse") } val (warehouse, metaStorePathAbsolute, hiveMetaStoreDB) = if (!metaRootPath.isEmpty) { @@ -71,12 +71,12 @@ object DSV2BenchmarkTest { val sessionBuilder = if (!configed) { val sessionBuilderTmp1 = sessionBuilderTmp - .master("local[3]") + .master("local[1]") .config("spark.driver.memory", "4G") .config("spark.driver.memoryOverhead", "6G") .config("spark.serializer", "org.apache.spark.serializer.JavaSerializer") .config("spark.default.parallelism", 1) - .config("spark.sql.shuffle.partitions", 1) + .config("spark.sql.shuffle.partitions",1) .config("spark.sql.adaptive.enabled", "false") .config("spark.sql.files.maxPartitionBytes", 1024 << 10 << 10) // default is 128M .config("spark.sql.files.minPartitionNum", "1") @@ -95,18 +95,20 @@ object DSV2BenchmarkTest { .config("spark.databricks.delta.snapshotPartitions", 1) .config("spark.databricks.delta.properties.defaults.checkpointInterval", 5) .config("spark.databricks.delta.stalenessLimit", 3600 * 1000) - //.config("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .config("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") //.config("spark.sql.execution.arrow.maxRecordsPerBatch", "20000") .config("spark.gluten.sql.columnar.columnartorow", "false") .config(GlutenConfig.GLUTEN_LOAD_NATIVE, "true") .config(GlutenConfig.GLUTEN_LOAD_ARROW, "false") .config(GlutenConfig.GLUTEN_LIB_PATH, - "path_to_clickhouse_engine/libch.so") - .config("spark.gluten.sql.columnar.iterator", "false") + "/usr/local/clickhouse/lib/libch.so") + .config("spark.gluten.sql.columnar.iterator", "true") //.config("spark.sql.planChangeLog.level", "info") .config("spark.sql.columnVector.offheap.enabled", "true") .config("spark.memory.offHeap.enabled", "true") .config("spark.memory.offHeap.size", "6442450944") + .config("spark.io.compression.codec", "LZ4") + .config("spark.gluten.sql.columnar.backend.lib", "clickhouse") if (!warehouse.isEmpty) { sessionBuilderTmp1.config("spark.sql.warehouse.dir", warehouse) @@ -126,14 +128,14 @@ object DSV2BenchmarkTest { val createTbl = true if (createTbl) { - createClickHouseTables(spark, parquetFilesPath, fileFormat) +// createClickHouseTables(spark, parquetFilesPath, fileFormat) createLocationClickHouseTable(spark) } val refreshTable = true if (refreshTable) { refreshClickHouseTable(spark) } - selectClickHouseTable(spark, executedCnt, sqlStr.mkString) +// selectClickHouseTable(spark, executedCnt, sqlStr.mkString) selectLocationClickHouseTable(spark, executedCnt, sqlStr.mkString) System.out.println("waiting for finishing") @@ -247,7 +249,7 @@ object DSV2BenchmarkTest { | USING clickhouse | TBLPROPERTIES (engine='MergeTree' | ) - | LOCATION '/data1/gluten-warehouse/ch_clickhouse' + | LOCATION '/home/saber/Documents/data/mergetree' |""".stripMargin) spark.sql( @@ -282,15 +284,27 @@ object DSV2BenchmarkTest { val startTime = System.nanoTime() spark.sql( s""" - |SELECT - | sum(l_extendedprice * l_discount) AS revenue - |FROM - | ${tableName} - |WHERE - | l_shipdate >= date'1994-01-01' - | AND l_shipdate < date'1994-01-01' + interval 1 year - | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 - | AND l_quantity < 24; + |SELECT + | l_returnflag, + | l_linestatus, + | sum(l_quantity) as sum_qty, + | sum(l_extendedprice) as sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, + | avg(l_quantity) as avg_qty, + | avg(l_extendedprice) as avg_price, + | avg(l_discount) as avg_disc, + | count(*) as count_order + |FROM + | ${tableName} + |WHERE + | l_shipdate <= date '1998-12-01' - interval '90' day + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; |""".stripMargin).show(200, false) val tookTime = (System.nanoTime() - startTime) / 1000000 println(s"Execute ${i} time, time: ${tookTime}") @@ -313,14 +327,26 @@ object DSV2BenchmarkTest { spark.sql( s""" |SELECT - | sum(l_extendedprice * l_discount) AS revenue + | l_returnflag, + | l_linestatus, + | sum(l_quantity) as sum_qty, + | sum(l_extendedprice) as sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, + | avg(l_quantity) as avg_qty, + | avg(l_extendedprice) as avg_price, + | avg(l_discount) as avg_disc, + | count(*) as count_order |FROM | ch_clickhouse |WHERE - | l_shipdate >= date'1994-01-01' - | AND l_shipdate < date'1994-01-01' + interval 1 year - | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 - | AND l_quantity < 24; + | l_shipdate <= date '1998-12-01' - interval '90' day + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; |""".stripMargin).show(200, false) val tookTime = (System.nanoTime() - startTime) / 1000000 println(s"Execute ${i} time, time: ${tookTime}") diff --git a/backends-velox/pom.xml b/backends-velox/pom.xml new file mode 100644 index 000000000000..53a944b5be2b --- /dev/null +++ b/backends-velox/pom.xml @@ -0,0 +1,439 @@ + + + + gluten-parent + io.glutenproject + 1.0.0-snapshot + + 4.0.0 + + backends-velox + jar + Gluten-Backends-Velox + + + 8 + 8 + ../cpp/ + ../cpp/build/releases/ + ${project.basedir}/src/main/resources + OFF + ${project.basedir}/../tools/build/arrow_install + + + + + org.apache.spark + spark-sql_${scala.binary.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + provided + + + org.apache.spark + spark-catalyst_${scala.binary.version} + provided + + + org.apache.spark + spark-sql_${scala.binary.version} + provided + + + org.apache.spark + spark-hive-thriftserver_${scala.binary.version} + provided + + + io.glutenproject + spark-sql-columnar-shims-common + ${project.version} + compile + + + io.glutenproject + gluten-jvm + ${project.version} + compile + + + org.scalacheck + scalacheck_${scala.binary.version} + 1.13.5 + test + + + org.scala-lang + scala-library + ${scala.version} + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.mockito + mockito-core + 2.23.4 + test + + + junit + junit + 4.12 + test + + + com.fasterxml.jackson.core + jackson-core + 2.10.0 + provided + + + com.fasterxml.jackson.core + jackson-annotations + 2.10.0 + provided + + + com.fasterxml.jackson.core + jackson-databind + 2.10.0 + provided + + + org.scalatestplus + scalatestplus-mockito_2.12 + 1.0.0-M2 + test + + + org.scalatestplus + scalatestplus-scalacheck_2.12 + 3.1.0.0-RC2 + test + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + provided + + + commons-io + commons-io + 2.11.0 + provided + + + + org.apache.arrow + ${arrow-memory.artifact} + ${arrow.version} + runtime + + + org.apache.arrow + arrow-memory-core + ${arrow.version} + + + io.netty + netty-common + + + io.netty + netty-buffer + + + + + + org.apache.arrow + arrow-vector + ${arrow.version} + + + io.netty + netty-common + + + io.netty + netty-buffer + + + + + + org.apache.arrow.gandiva + arrow-gandiva + ${arrow.version} + + + io.netty + netty-common + + + io.netty + netty-buffer + + + protobuf-java + com.google.protobuf + + + + + + 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 + + + protobuf-java + com.google.protobuf + + + compile + + + + + + + ${cpp.build.dir} + + + ${resource.dir} + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + exec-maven-plugin + org.codehaus.mojo + 1.6.0 + + + Build arrow + generate-resources + + exec + + + bash + + ${arrow.script.dir}/build_arrow.sh + --tests=${cpp_tests} + --build_arrow=${build_arrow} + --static_arrow=${static_arrow} + --arrow_root=${arrow_root} + + + + + Build cpp + generate-resources + + exec + + + bash + + ${cpp.dir}/compile.sh + ${build_cpp} + ${cpp_tests} + ${jvm.build_arrow} + ${static_arrow} + ${build_protobuf} + ${arrow_root} + ${jvm.arrow.bfs.install.dir} + ${build_jemalloc} + ${build_gazelle_cpp} + ${build_velox} + ${velox_home} + + + + + Build velox + generate-resources + + exec + + + bash + + ${velox.script.dir}/build_velox.sh + --build_velox_from_source=${build_velox_from_source} + + + + + + + + org.apache.maven.plugins + maven-resources-plugin + 3.0.1 + + + net.alchim31.maven + scala-maven-plugin + 4.3.0 + + ${scala.recompile.mode} + + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.8.0 + + 1.8 + 1.8 + + + + compile + + compile + + + + + + maven-assembly-plugin + 3.3.0 + + ${jar.assembly.name.prefix}-${project.version} + + jar-with-dependencies + + + + + make-assembly + package + + single + + + + + + org.scalastyle + scalastyle-maven-plugin + 1.0.0 + + false + false + true + false + ${project.basedir}/src/main/scala + ${project.basedir}/src/test/scala + ${project.basedir}/../scalastyle-config.xml + ${project.basedir}/scalastyle-output.xml + UTF-8 + + + + + check + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 3.1.1 + + false + true + + ${project.basedir}/src/main/java + ${project.basedir}/src/main/scala + + + ${project.basedir}/src/test/java + + ${project.basedir}/../dev/checkstyle.xml + ${project.basedir}/target/checkstyle-output.xml + UTF-8 + UTF-8 + + + + com.puppycrawl.tools + checkstyle + 8.29 + + + + + + check + + + + + + org.scalatest + scalatest-maven-plugin + + true + . + + + + test + + test + + + + + + + \ No newline at end of file diff --git a/jvm/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java b/backends-velox/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java similarity index 57% rename from jvm/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java rename to backends-velox/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java index 47be776ee895..6ba5b2236e2c 100644 --- a/jvm/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java +++ b/backends-velox/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java @@ -17,52 +17,24 @@ package io.glutenproject.execution; -import io.glutenproject.expression.ConverterUtils; +import io.glutenproject.expression.ArrowConverterUtils; import org.apache.arrow.dataset.jni.UnsafeRecordBatchSerializer; import org.apache.arrow.vector.ipc.message.ArrowRecordBatch; import org.apache.spark.sql.vectorized.ColumnarBatch; import java.util.Iterator; -/** - * Iterate on flatbuffers-serialized {@link org.apache.arrow.vector.ipc.message.ArrowRecordBatch}. - *

- * {@link #next()} should be called from C++ scanner to read Java-generated Arrow data. - */ -interface NativeSerializedRecordBatchIterator extends Iterator, AutoCloseable { - - /** - * Return next serialized {@link org.apache.arrow.vector.ipc.message.ArrowRecordBatch} Java - * byte array. - */ - @Override - byte[] next(); -} - -public class ColumnarNativeIterator implements NativeSerializedRecordBatchIterator { - protected final Iterator delegated; - protected ColumnarBatch nextBatch = null; +public class ColumnarNativeIterator extends AbstractColumnarNativeIterator { public ColumnarNativeIterator(Iterator delegated) { - this.delegated = delegated; - } - - @Override - public boolean hasNext() { - while (delegated.hasNext()) { - nextBatch = delegated.next(); - if (nextBatch.numRows() > 0) { - return true; - } - } - return false; + super(delegated); } @Override public byte[] next() { ColumnarBatch dep_cb = nextBatch; if (dep_cb.numRows() > 0) { - ArrowRecordBatch dep_rb = ConverterUtils.createArrowRecordBatch(dep_cb); + ArrowRecordBatch dep_rb = ArrowConverterUtils.createArrowRecordBatch(dep_cb); return serialize(dep_rb); } else { throw new IllegalStateException(); @@ -72,9 +44,4 @@ public byte[] next() { private byte[] serialize(ArrowRecordBatch batch) { return UnsafeRecordBatchSerializer.serializeUnsafe(batch); } - - @Override - public void close() throws Exception { - - } } diff --git a/backends-velox/src/main/java/io/glutenproject/execution/VeloxNativeColumnarToRowExec.scala b/backends-velox/src/main/java/io/glutenproject/execution/VeloxNativeColumnarToRowExec.scala new file mode 100644 index 000000000000..5b2929f94aa5 --- /dev/null +++ b/backends-velox/src/main/java/io/glutenproject/execution/VeloxNativeColumnarToRowExec.scala @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.execution + +import io.glutenproject.GlutenConfig +import io.glutenproject.expression.ArrowConverterUtils +import io.glutenproject.vectorized.{ArrowWritableColumnVector, NativeColumnarToRowJniWrapper} + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ListBuffer +import scala.concurrent.duration._ + +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} +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils +import org.apache.spark.sql.types._ + +class VeloxNativeColumnarToRowExec(child: SparkPlan) + extends NativeColumnarToRowExec(child = child) { + override def nodeName: String = "VeloxNativeColumnarToRowExec" + + override def supportCodegen: Boolean = false + + override def buildCheck(): Unit = { + val schema = child.schema + for (field <- schema.fields) { + field.dataType match { + case d: BooleanType => + case d: ByteType => + case d: ShortType => + case d: IntegerType => + case d: LongType => + case d: FloatType => + case d: DoubleType => + case d: StringType => + case d: DateType => + case d: DecimalType => + case d: TimestampType => + case d: BinaryType => + case _ => + throw new UnsupportedOperationException(s"${field.dataType} is not supported in NativeColumnarToRowExec.") + } + } + } + + override def doExecuteInternal(): RDD[InternalRow] = { + val numOutputRows = longMetric("numOutputRows") + val numInputBatches = longMetric("numInputBatches") + val convertTime = longMetric("convertTime") + + // Fake Arrow format will be returned for WS transformer. + // TODO: use a Velox layer on the top of the base layer. + if (GlutenConfig.getConf.isVeloxBackend) { + if (child.isInstanceOf[WholeStageTransformerExec]) { + child.asInstanceOf[WholeStageTransformerExec].setFakeOutput() + } + } + + child.executeColumnar().mapPartitions { batches => + // TODO:: pass the jni jniWrapper and arrowSchema and serializeSchema method by broadcast + val jniWrapper = new NativeColumnarToRowJniWrapper() + var arrowSchema: Array[Byte] = null + + def serializeSchema(fields: Seq[Field]): Array[Byte] = { + val schema = new Schema(fields.asJava) + ArrowConverterUtils.getSchemaBytesBuf(schema) + } + + batches.flatMap { batch => + numInputBatches += 1 + numOutputRows += batch.numRows() + + if (batch.numRows == 0) { + logInfo(s"Skip ColumnarBatch of ${batch.numRows} rows, ${batch.numCols} cols") + Iterator.empty + } else if (this.output.isEmpty || (batch.numCols() > 0 && + !batch.column(0).isInstanceOf[ArrowWritableColumnVector])) { + // Fallback to ColumnarToRow + val localOutput = this.output + numInputBatches += 1 + numOutputRows += batch.numRows() + + val toUnsafe = UnsafeProjection.create(localOutput, localOutput) + batch.rowIterator().asScala.map(toUnsafe) + } else { + val bufAddrs = new ListBuffer[Long]() + val bufSizes = new ListBuffer[Long]() + val fields = new ListBuffer[Field]() + (0 until batch.numCols).foreach { idx => + val column = batch.column(idx).asInstanceOf[ArrowWritableColumnVector] + fields += column.getValueVector.getField + column.getValueVector.getBuffers(false) + .foreach { buffer => + bufAddrs += buffer.memoryAddress() + try { + bufSizes += buffer.readableBytes() + } catch { + case e: Throwable => + // For Velox, the returned format is faked arrow format, + // and the offset buffer is invalid. Only the buffer address is cared. + if (GlutenConfig.getConf.isVeloxBackend && + child.output(idx).dataType == StringType) { + // Add a fake value here for String column. + bufSizes += 1 + } else { + throw e + } + } + } + } + + if (arrowSchema == null) { + arrowSchema = serializeSchema(fields) + } + + val beforeConvert = System.nanoTime() + + val info = jniWrapper.nativeConvertColumnarToRow( + arrowSchema, batch.numRows, bufAddrs.toArray, bufSizes.toArray, + SparkMemoryUtils.contextMemoryPool().getNativeInstanceId, wsChild) + + convertTime += NANOSECONDS.toMillis(System.nanoTime() - beforeConvert) + + new Iterator[InternalRow] { + var rowId = 0 + val row = new UnsafeRow(batch.numCols()) + var closed = false + override def hasNext: Boolean = { + val result = rowId < batch.numRows() + if (!result && !closed) { + jniWrapper.nativeClose(info.instanceID) + closed = true + } + return result + } + + override def next: UnsafeRow = { + if (rowId >= batch.numRows()) throw new NoSuchElementException + + val (offset, length) = (info.offsets(rowId), info.lengths(rowId)) + row.pointTo(null, info.memoryAddress + offset, length.toInt) + rowId += 1 + row + } + } + } + } + } + } + + override def canEqual(other: Any): Boolean = other.isInstanceOf[VeloxNativeColumnarToRowExec] + + override def equals(other: Any): Boolean = other match { + case that: VeloxNativeColumnarToRowExec => + (that canEqual this) && super.equals(that) + case _ => false + } +} + diff --git a/backends-velox/src/main/java/io/glutenproject/execution/VeloxRowToArrowColumnarExec.scala b/backends-velox/src/main/java/io/glutenproject/execution/VeloxRowToArrowColumnarExec.scala new file mode 100644 index 000000000000..4178d4866f01 --- /dev/null +++ b/backends-velox/src/main/java/io/glutenproject/execution/VeloxRowToArrowColumnarExec.scala @@ -0,0 +1,293 @@ +/* + * 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 io.glutenproject.execution + +import java.util.concurrent.TimeUnit._ + +import io.glutenproject.vectorized._ + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.SpecializedGetters +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils.UnsafeItr +import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, WritableColumnVector} +import org.apache.spark.sql.types._ +import org.apache.spark.sql.vectorized.ColumnarBatch + +class RowToColumnConverter(schema: StructType) extends Serializable { + private val converters = schema.fields.map { + f => RowToColumnConverter.getConverterForType(f.dataType, f.nullable) + } + + final def convert(row: InternalRow, vectors: Array[WritableColumnVector]): Unit = { + var idx = 0 + while (idx < row.numFields) { + converters(idx).append(row, idx, vectors(idx)) + idx += 1 + } + } +} + +object RowToColumnConverter { + private abstract class TypeConverter extends Serializable { + def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit + } + + private final case class BasicNullableTypeConverter(base: TypeConverter) extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + if (row.isNullAt(column)) { + cv.appendNull + } else { + base.append(row, column, cv) + } + } + } + + private final case class StructNullableTypeConverter(base: TypeConverter) extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + if (row.isNullAt(column)) { + cv.appendStruct(true) + } else { + base.append(row, column, cv) + } + } + } + + private def getConverterForType(dataType: DataType, nullable: Boolean): TypeConverter = { + val core = dataType match { + case BooleanType => BooleanConverter + case ByteType => ByteConverter + case ShortType => ShortConverter + case IntegerType | DateType => IntConverter + case FloatType => FloatConverter + case LongType | TimestampType => LongConverter + case DoubleType => DoubleConverter + case StringType => StringConverter + case BinaryType => BinaryConverter + case CalendarIntervalType => CalendarConverter + case at: ArrayType => new ArrayConverter(getConverterForType(at.elementType, nullable)) + case st: StructType => new StructConverter(st.fields.map( + (f) => getConverterForType(f.dataType, f.nullable))) + case dt: DecimalType => new DecimalConverter(dt) + case mt: MapType => new MapConverter(getConverterForType(mt.keyType, nullable), + getConverterForType(mt.valueType, nullable)) + case unknown => throw new UnsupportedOperationException( + s"Type $unknown not supported") + } + + if (nullable) { + dataType match { + case CalendarIntervalType => new StructNullableTypeConverter(core) + case st: StructType => new StructNullableTypeConverter(core) + case _ => new BasicNullableTypeConverter(core) + } + } else { + core + } + } + + private object BooleanConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = + cv.appendBoolean(row.getBoolean(column)) + } + + private object ByteConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = + cv.appendByte(row.getByte(column)) + } + + private object ShortConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = + cv.appendShort(row.getShort(column)) + } + + private object IntConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = + cv.appendInt(row.getInt(column)) + } + + private object FloatConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = + cv.appendFloat(row.getFloat(column)) + } + + private object LongConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = + cv.appendLong(row.getLong(column)) + } + + private object DoubleConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = + cv.appendDouble(row.getDouble(column)) + } + + private object StringConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + val data = row.getUTF8String(column).getBytes + cv.asInstanceOf[ArrowWritableColumnVector].appendString(data, 0, data.length) + } + } + + private object BinaryConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + val data = row.getBinary(column) + cv.asInstanceOf[ArrowWritableColumnVector].appendString(data, 0, data.length) + } + } + + private object CalendarConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + val c = row.getInterval(column) + cv.appendStruct(false) + cv.getChild(0).appendInt(c.months) + cv.getChild(1).appendInt(c.days) + cv.getChild(2).appendLong(c.microseconds) + } + } + + private case class ArrayConverter(childConverter: TypeConverter) extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + val values = row.getArray(column) + val numElements = values.numElements() + cv.appendArray(numElements) + val arrData = cv.arrayData() + for (i <- 0 until numElements) { + childConverter.append(values, i, arrData) + } + } + } + + private case class StructConverter(childConverters: Array[TypeConverter]) extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + cv.appendStruct(false) + val data = row.getStruct(column, childConverters.length) + for (i <- 0 until childConverters.length) { + childConverters(i).append(data, i, cv.getChild(i)) + } + } + } + + private case class DecimalConverter(dt: DecimalType) extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + val d = row.getDecimal(column, dt.precision, dt.scale) + if (dt.precision <= Decimal.MAX_INT_DIGITS) { + cv.appendInt(d.toUnscaledLong.toInt) + } else if (dt.precision <= Decimal.MAX_LONG_DIGITS) { + cv.appendLong(d.toUnscaledLong) + } else { + val value = d.toJavaBigDecimal + cv.asInstanceOf[ArrowWritableColumnVector].appendDecimal(value) + } + } + } + + private case class MapConverter(keyConverter: TypeConverter, valueConverter: TypeConverter) + extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + val m = row.getMap(column) + val keys = cv.getChild(0) + val values = cv.getChild(1) + val numElements = m.numElements() + cv.appendArray(numElements) + + val srcKeys = m.keyArray() + val srcValues = m.valueArray() + + for (i <- 0 until numElements) { + keyConverter.append(srcKeys, i, keys) + valueConverter.append(srcValues, i, values) + } + } + } +} + +/** + * Provides a common executor to translate an [[RDD]] of [[InternalRow]] into an [[RDD]] of + * [[ColumnarBatch]]. This is inserted whenever such a transition is determined to be needed. + * + * This is similar to some of the code in ArrowConverters.scala and + * [[org.apache.spark.sql.execution.arrow.ArrowWriter]]. That code is more specialized + * to convert [[InternalRow]] to Arrow formatted data, but in the future if we make + * [[OffHeapColumnVector]] internally Arrow formatted we may be able to replace much of that code. + * + * This is also similar to + * [[org.apache.spark.sql.execution.vectorized.ColumnVectorUtils.populate()]] and + * [[org.apache.spark.sql.execution.vectorized.ColumnVectorUtils.toBatch()]] toBatch is only ever + * called from tests and can probably be removed, but populate is used by both Orc and Parquet + * to initialize partition and missing columns. There is some chance that we could replace + * populate with [[RowToColumnConverter]], but the performance requirements are different and it + * would only be to reduce code. + */ +class VeloxRowToArrowColumnarExec(child: SparkPlan) extends RowToArrowColumnarExec(child = child) { + + override def doExecuteColumnarInternal(): RDD[ColumnarBatch] = { + val numInputRows = longMetric("numInputRows") + val numOutputBatches = longMetric("numOutputBatches") + val processTime = longMetric("processTime") + // Instead of creating a new config we are reusing columnBatchSize. In the future if we do + // combine with some of the Arrow conversion tools we will need to unify some of the configs. + val numRows = conf.columnBatchSize + // This avoids calling `schema` in the RDD closure, so that we don't need to include the entire + // plan (this) in the closure. + val localSchema = this.schema + child.execute().mapPartitions { rowIterator => + if (rowIterator.hasNext) { + val res = new Iterator[ColumnarBatch] { + private val converters = new RowToColumnConverter(localSchema) + private var last_cb: ColumnarBatch = null + private var elapse: Long = 0 + + override def hasNext: Boolean = { + rowIterator.hasNext + } + + override def next(): ColumnarBatch = { + val vectors: Seq[WritableColumnVector] = + ArrowWritableColumnVector.allocateColumns(numRows, schema) + var rowCount = 0 + while (rowCount < numRows && rowIterator.hasNext) { + val row = rowIterator.next() + val start = System.nanoTime() + converters.convert(row, vectors.toArray) + elapse += System.nanoTime() - start + rowCount += 1 + } + vectors.foreach(v => v.asInstanceOf[ArrowWritableColumnVector].setValueCount(rowCount)) + processTime.set(NANOSECONDS.toMillis(elapse)) + numInputRows += rowCount + numOutputBatches += 1 + last_cb = new ColumnarBatch(vectors.toArray, rowCount) + last_cb + } + } + new UnsafeItr(res) + } else { + Iterator.empty + } + } + } + + override def canEqual(other: Any): Boolean = other.isInstanceOf[VeloxRowToArrowColumnarExec] + + override def equals(other: Any): Boolean = other match { + case that: VeloxRowToArrowColumnarExec => + (that canEqual this) && super.equals(that) + case _ => false + } +} diff --git a/jvm/src/main/java/io/glutenproject/expression/ArrowColumnarBatch.java b/backends-velox/src/main/java/io/glutenproject/expression/ArrowColumnarBatch.java similarity index 86% rename from jvm/src/main/java/io/glutenproject/expression/ArrowColumnarBatch.java rename to backends-velox/src/main/java/io/glutenproject/expression/ArrowColumnarBatch.java index a52e131dba5c..1cb30c8a04f4 100644 --- a/jvm/src/main/java/io/glutenproject/expression/ArrowColumnarBatch.java +++ b/backends-velox/src/main/java/io/glutenproject/expression/ArrowColumnarBatch.java @@ -17,14 +17,11 @@ package io.glutenproject.expression; -import java.io.*; -import org.apache.arrow.memory.ArrowBuf; -import java.util.ArrayList; -import java.util.List; -import org.apache.arrow.vector.ipc.message.ArrowRecordBatch; -import org.apache.arrow.vector.ipc.message.ArrowFieldNode; import org.apache.arrow.vector.ValueVector; +import java.io.IOException; +import java.util.List; + public class ArrowColumnarBatch implements AutoCloseable { int fieldsNum; diff --git a/jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryConsumer.java b/backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryConsumer.java similarity index 100% rename from jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryConsumer.java rename to backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryConsumer.java diff --git a/jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryMetrics.java b/backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryMetrics.java similarity index 100% rename from jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryMetrics.java rename to backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryMetrics.java diff --git a/jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/SparkManagedAllocationListener.java b/backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/SparkManagedAllocationListener.java similarity index 100% rename from jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/SparkManagedAllocationListener.java rename to backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/SparkManagedAllocationListener.java diff --git a/jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/SparkManagedReservationListener.java b/backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/SparkManagedReservationListener.java similarity index 100% rename from jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/SparkManagedReservationListener.java rename to backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/SparkManagedReservationListener.java diff --git a/jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/Spiller.java b/backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/Spiller.java similarity index 100% rename from jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/Spiller.java rename to backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/Spiller.java diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ArrowBufBuilder.java b/backends-velox/src/main/java/io/glutenproject/vectorized/ArrowBufBuilder.java similarity index 100% rename from jvm/src/main/java/io/glutenproject/vectorized/ArrowBufBuilder.java rename to backends-velox/src/main/java/io/glutenproject/vectorized/ArrowBufBuilder.java diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ArrowCompressedStreamReader.java b/backends-velox/src/main/java/io/glutenproject/vectorized/ArrowCompressedStreamReader.java similarity index 100% rename from jvm/src/main/java/io/glutenproject/vectorized/ArrowCompressedStreamReader.java rename to backends-velox/src/main/java/io/glutenproject/vectorized/ArrowCompressedStreamReader.java diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ArrowFieldNodeBuilder.java b/backends-velox/src/main/java/io/glutenproject/vectorized/ArrowFieldNodeBuilder.java similarity index 100% rename from jvm/src/main/java/io/glutenproject/vectorized/ArrowFieldNodeBuilder.java rename to backends-velox/src/main/java/io/glutenproject/vectorized/ArrowFieldNodeBuilder.java diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ArrowWritableColumnVector.java b/backends-velox/src/main/java/io/glutenproject/vectorized/ArrowWritableColumnVector.java similarity index 100% rename from jvm/src/main/java/io/glutenproject/vectorized/ArrowWritableColumnVector.java rename to backends-velox/src/main/java/io/glutenproject/vectorized/ArrowWritableColumnVector.java diff --git a/jvm/src/main/java/io/glutenproject/vectorized/BatchIterator.java b/backends-velox/src/main/java/io/glutenproject/vectorized/BatchIterator.java similarity index 61% rename from jvm/src/main/java/io/glutenproject/vectorized/BatchIterator.java rename to backends-velox/src/main/java/io/glutenproject/vectorized/BatchIterator.java index 4cc5866e8e02..4699f46b7ed1 100644 --- a/jvm/src/main/java/io/glutenproject/vectorized/BatchIterator.java +++ b/backends-velox/src/main/java/io/glutenproject/vectorized/BatchIterator.java @@ -17,44 +17,37 @@ package io.glutenproject.vectorized; -import java.io.IOException; - import org.apache.arrow.dataset.jni.UnsafeRecordBatchSerializer; import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.vector.ipc.message.ArrowRecordBatch; -import org.apache.arrow.vector.types.pojo.Schema; - -import java.io.Serializable; -import java.io.ByteArrayOutputStream; -import java.nio.channels.Channels; -import org.apache.arrow.vector.ipc.WriteChannel; -import org.apache.arrow.vector.ipc.message.MessageSerializer; import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarBatch; -public class BatchIterator implements AutoCloseable, Serializable { +import java.io.IOException; +import java.io.Serializable; + +public class BatchIterator extends AbstractBatchIterator { private native boolean nativeHasNext(long nativeHandler); private native byte[] nativeNext(long nativeHandler); private native long nativeCHNext(long nativeHandler); private native void nativeClose(long nativeHandler); private native MetricsObject nativeFetchMetrics(long nativeHandler); - private long nativeHandler = 0; - private boolean closed = false; - public BatchIterator() throws IOException {} public BatchIterator(long instance_id) throws IOException { - nativeHandler = instance_id; + super(instance_id); } - public boolean hasNext() throws IOException { + @Override + public boolean hasNextInternal() throws IOException { return nativeHasNext(nativeHandler); } - public ArrowRecordBatch next() throws IOException { + @Override + public ArrowRecordBatch nextInternal() throws IOException { BufferAllocator allocator = SparkMemoryUtils.contextAllocator(); if (nativeHandler == 0) { return null; @@ -67,39 +60,16 @@ public ArrowRecordBatch next() throws IOException { serializedRecordBatch); } - public MetricsObject getMetrics() throws IOException, ClassNotFoundException { - if (nativeHandler == 0) { - return null; - } + @Override + public MetricsObject getMetricsInternal() throws IOException, ClassNotFoundException { return nativeFetchMetrics(nativeHandler); } - public ColumnarBatch chNext() { - long block = nativeCHNext(nativeHandler); - CHNativeBlock nativeBlock = new CHNativeBlock(block); - int cols = nativeBlock.numColumns(); - ColumnVector[] columnVectors = new ColumnVector[cols]; - for (int i = 0; i < cols; i++) { - columnVectors[i] = new CHColumnVector(DataType.fromDDL(nativeBlock.getTypeByPosition(i)), block, i); - } - return new ColumnarBatch(columnVectors, nativeBlock.numRows()); - } - @Override - public void close() { + public void closeInternal() { if (!closed) { nativeClose(nativeHandler); closed = true; } } - - byte[] getSchemaBytesBuf(Schema schema) throws IOException { - ByteArrayOutputStream out = new ByteArrayOutputStream(); - MessageSerializer.serialize(new WriteChannel(Channels.newChannel(out)), schema); - return out.toByteArray(); - } - - long getInstanceId() { - return nativeHandler; - } } diff --git a/jvm/src/main/java/io/glutenproject/vectorized/CompressedVectorLoader.java b/backends-velox/src/main/java/io/glutenproject/vectorized/CompressedVectorLoader.java similarity index 99% rename from jvm/src/main/java/io/glutenproject/vectorized/CompressedVectorLoader.java rename to backends-velox/src/main/java/io/glutenproject/vectorized/CompressedVectorLoader.java index 18707af95a7d..8f6288f145ec 100644 --- a/jvm/src/main/java/io/glutenproject/vectorized/CompressedVectorLoader.java +++ b/backends-velox/src/main/java/io/glutenproject/vectorized/CompressedVectorLoader.java @@ -17,8 +17,7 @@ package io.glutenproject.vectorized; -import java.util.Iterator; - +import org.apache.arrow.memory.ArrowBuf; import org.apache.arrow.util.Collections2; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.VectorLoader; @@ -26,7 +25,7 @@ import org.apache.arrow.vector.ipc.message.ArrowFieldNode; import org.apache.arrow.vector.ipc.message.ArrowRecordBatch; -import org.apache.arrow.memory.ArrowBuf; +import java.util.Iterator; /** Loads compressed buffers into vectors. */ public class CompressedVectorLoader extends VectorLoader { diff --git a/jvm/src/main/java/io/glutenproject/vectorized/SchemaAwareArrowCompressedStreamReader.java b/backends-velox/src/main/java/io/glutenproject/vectorized/SchemaAwareArrowCompressedStreamReader.java similarity index 100% rename from jvm/src/main/java/io/glutenproject/vectorized/SchemaAwareArrowCompressedStreamReader.java rename to backends-velox/src/main/java/io/glutenproject/vectorized/SchemaAwareArrowCompressedStreamReader.java diff --git a/jvm/src/main/java/io/glutenproject/vectorized/SchemaAwareArrowStreamReader.java b/backends-velox/src/main/java/io/glutenproject/vectorized/SchemaAwareArrowStreamReader.java similarity index 100% rename from jvm/src/main/java/io/glutenproject/vectorized/SchemaAwareArrowStreamReader.java rename to backends-velox/src/main/java/io/glutenproject/vectorized/SchemaAwareArrowStreamReader.java diff --git a/jvm/src/main/java/io/glutenproject/vectorized/SerializableObject.java b/backends-velox/src/main/java/io/glutenproject/vectorized/SerializableObject.java similarity index 97% rename from jvm/src/main/java/io/glutenproject/vectorized/SerializableObject.java rename to backends-velox/src/main/java/io/glutenproject/vectorized/SerializableObject.java index 9084eb41880e..b96d17f8d776 100644 --- a/jvm/src/main/java/io/glutenproject/vectorized/SerializableObject.java +++ b/backends-velox/src/main/java/io/glutenproject/vectorized/SerializableObject.java @@ -17,7 +17,6 @@ package io.glutenproject.vectorized; -import java.io.*; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.KryoSerializable; import com.esotericsoftware.kryo.io.Input; @@ -30,6 +29,12 @@ import io.netty.buffer.UnpooledByteBufAllocator; import org.apache.arrow.util.AutoCloseables; +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.io.OutputStream; + /** ArrowBufBuilder. */ public class SerializableObject implements Externalizable, KryoSerializable { public int total_size; diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ShuffleDecompressionJniWrapper.java b/backends-velox/src/main/java/io/glutenproject/vectorized/ShuffleDecompressionJniWrapper.java similarity index 100% rename from jvm/src/main/java/io/glutenproject/vectorized/ShuffleDecompressionJniWrapper.java rename to backends-velox/src/main/java/io/glutenproject/vectorized/ShuffleDecompressionJniWrapper.java diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ShuffleSplitterJniWrapper.java b/backends-velox/src/main/java/io/glutenproject/vectorized/ShuffleSplitterJniWrapper.java similarity index 100% rename from jvm/src/main/java/io/glutenproject/vectorized/ShuffleSplitterJniWrapper.java rename to backends-velox/src/main/java/io/glutenproject/vectorized/ShuffleSplitterJniWrapper.java diff --git a/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.IIteratorApi b/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.IIteratorApi new file mode 100644 index 000000000000..afdecf215121 --- /dev/null +++ b/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.IIteratorApi @@ -0,0 +1 @@ +io.glutenproject.backendsapi.velox.VeloxIteratorApi \ No newline at end of file diff --git a/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ISparkPlanExecApi b/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ISparkPlanExecApi new file mode 100644 index 000000000000..b11a3a9bbe4e --- /dev/null +++ b/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ISparkPlanExecApi @@ -0,0 +1 @@ +io.glutenproject.backendsapi.velox.VeloxSparkPlanExecApi \ No newline at end of file diff --git a/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ITransformerApi b/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ITransformerApi new file mode 100644 index 000000000000..0716759ad1e9 --- /dev/null +++ b/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ITransformerApi @@ -0,0 +1 @@ +io.glutenproject.backendsapi.velox.VeloxTransformerApi \ No newline at end of file diff --git a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxIteratorApi.scala b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxIteratorApi.scala new file mode 100644 index 000000000000..c74f3b392075 --- /dev/null +++ b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxIteratorApi.scala @@ -0,0 +1,381 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.backendsapi.velox + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ListBuffer + +import io.glutenproject.{GlutenConfig, GlutenNumaBindingInfo} +import io.glutenproject.backendsapi.IIteratorApi +import io.glutenproject.execution._ +import io.glutenproject.expression.ArrowConverterUtils +import io.glutenproject.substrait.plan.PlanNode +import io.glutenproject.substrait.rel.LocalFilesBuilder +import io.glutenproject.vectorized._ +import org.apache.arrow.vector.ipc.message.ArrowRecordBatch +import org.apache.arrow.vector.types.pojo.Schema +import org.apache.spark.{InterruptibleIterator, SparkConf, TaskContext} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.FilePartition +import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.util.{ExecutorManager, UserAddedJarUtils} + +class VeloxIteratorApi extends IIteratorApi with Logging { + + /** + * Generate native row partition. + * + * @return + */ + override def genNativeFilePartition(p: InputPartition, + wsCxt: WholestageTransformContext + ): BaseNativeFilePartition = { + p match { + case FilePartition(index, files) => + val paths = new java.util.ArrayList[String]() + val starts = new java.util.ArrayList[java.lang.Long]() + val lengths = new java.util.ArrayList[java.lang.Long]() + files.foreach { f => + paths.add(f.filePath) + starts.add(new java.lang.Long(f.start)) + lengths.add(new java.lang.Long(f.length)) + } + val localFilesNode = LocalFilesBuilder.makeLocalFiles(index, paths, starts, lengths) + wsCxt.substraitContext.setLocalFilesNode(localFilesNode) + val substraitPlan = wsCxt.root.toProtobuf + /* + val out = new DataOutputStream(new FileOutputStream("/tmp/SubStraitTest-Q6.dat", + false)); + out.write(substraitPlan.toByteArray()); + out.flush(); + */ + // logWarning(s"The substrait plan for partition ${index}:\n${substraitPlan.toString}") + NativeFilePartition(index, files, substraitPlan.toByteArray) + } + } + + + /** + * Generate Iterator[ColumnarBatch] for CoalesceBatchesExec. + * + * @param iter + * @param recordsPerBatch + * @param numOutputRows + * @param numInputBatches + * @param numOutputBatches + * @param collectTime + * @param concatTime + * @param avgCoalescedNumRows + * @return + */ + override def genCoalesceIterator(iter: Iterator[ColumnarBatch], + recordsPerBatch: Int, + numOutputRows: SQLMetric, + numInputBatches: SQLMetric, + numOutputBatches: SQLMetric, + collectTime: SQLMetric, + concatTime: SQLMetric, + avgCoalescedNumRows: SQLMetric): Iterator[ColumnarBatch] = { + import io.glutenproject.utils.VeloxImplicitClass._ + + val beforeInput = System.nanoTime + val hasInput = iter.hasNext + collectTime += System.nanoTime - beforeInput + val res = if (hasInput) { + new Iterator[ColumnarBatch] { + var numBatchesTotal: Long = _ + var numRowsTotal: Long = _ + SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit] { _ => + if (numBatchesTotal > 0) { + avgCoalescedNumRows.set(numRowsTotal.toDouble / numBatchesTotal) + } + } + + override def hasNext: Boolean = { + val beforeNext = System.nanoTime + val hasNext = iter.hasNext + collectTime += System.nanoTime - beforeNext + hasNext + } + + override def next(): ColumnarBatch = { + if (!hasNext) { + throw new NoSuchElementException("End of ColumnarBatch iterator") + } + + var rowCount = 0 + val batchesToAppend = ListBuffer[ColumnarBatch]() + + while (hasNext && rowCount < recordsPerBatch) { + val delta = iter.next() + delta.retain() + rowCount += delta.numRows + batchesToAppend += delta + } + + // chendi: We need make sure target FieldTypes are exactly the same as src + val expected_output_arrow_fields = if (batchesToAppend.size > 0) { + (0 until batchesToAppend(0).numCols).map(i => { + batchesToAppend(0).column(i).asInstanceOf[ArrowWritableColumnVector].getValueVector.getField + }) + } else { + Nil + } + + val resultStructType = ArrowUtils.fromArrowSchema(new Schema(expected_output_arrow_fields.asJava)) + val beforeConcat = System.nanoTime + val resultColumnVectors = + ArrowWritableColumnVector.allocateColumns(rowCount, resultStructType).toArray + val target = + new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), rowCount) + coalesce(target, batchesToAppend.toList) + target.setNumRows(rowCount) + + concatTime += System.nanoTime - beforeConcat + numOutputRows += rowCount + numInputBatches += batchesToAppend.length + numOutputBatches += 1 + + // used for calculating avgCoalescedNumRows + numRowsTotal += rowCount + numBatchesTotal += 1 + + batchesToAppend.foreach(cb => cb.close()) + + target + } + } + } else { + Iterator.empty + } + new CloseableColumnBatchIterator(res) + } + + + /** + * Generate closeable ColumnBatch iterator. + * + * @param iter + * @return + */ + override def genCloseableColumnBatchIterator(iter: Iterator[ColumnarBatch] + ): Iterator[ColumnarBatch] = { + new CloseableColumnBatchIterator(iter) + } + + /** + * Generate Iterator[ColumnarBatch] for first stage. + * + * @return + */ + override def genFirstStageIterator(inputPartition: BaseNativeFilePartition, + loadNative: Boolean, + outputAttributes: Seq[Attribute], + context: TaskContext, + jarList: Seq[String]): Iterator[ColumnarBatch] = { + import org.apache.spark.sql.util.OASPackageBridge._ + var inputSchema : Schema = null + var outputSchema : Schema = null + var resIter : AbstractBatchIterator = null + if (loadNative) { + // TODO: 'jarList' is kept for codegen + val transKernel = new ExpressionEvaluator(jarList.asJava) + val inBatchIters = new java.util.ArrayList[AbstractColumnarNativeIterator]() + outputSchema = ArrowConverterUtils.toArrowSchema(outputAttributes) + resIter = transKernel.createKernelWithBatchIterator( + inputPartition.substraitPlan, inBatchIters) + SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit] { _ => resIter.close() } + } + val iter = new Iterator[Any] { + private val inputMetrics = TaskContext.get().taskMetrics().inputMetrics + + override def hasNext: Boolean = { + if (loadNative) { + resIter.hasNext + } else { + false + } + } + + override def next(): Any = { + if (!hasNext) { + throw new java.util.NoSuchElementException("End of stream") + } + val rb = resIter.next().asInstanceOf[ArrowRecordBatch] + if (rb == null) { + val resultStructType = ArrowUtils.fromArrowSchema(outputSchema) + val resultColumnVectors = + ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray + return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0) + } + val outputNumRows = rb.getLength + val output = ArrowConverterUtils.fromArrowRecordBatch(outputSchema, rb) + ArrowConverterUtils.releaseArrowRecordBatch(rb) + val cb = new ColumnarBatch(output.map(v => v.asInstanceOf[ColumnVector]), outputNumRows) + val bytes: Long = cb 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) + cb + } + } + + // TODO: SPARK-25083 remove the type erasure hack in data source scan + new InterruptibleIterator(context, + new CloseableColumnBatchIterator(iter.asInstanceOf[Iterator[ColumnarBatch]])) + } + + /** + * Generate Iterator[ColumnarBatch] for final stage. + * + * @return + */ + override def genFinalStageIterator(iter: Iterator[ColumnarBatch], + numaBindingInfo: GlutenNumaBindingInfo, + listJars: Seq[String], + signature: String, + sparkConf: SparkConf, + outputAttributes: Seq[Attribute], + rootNode: PlanNode, + streamedSortPlan: SparkPlan, + pipelineTime: SQLMetric, + buildRelationBatchHolder: Seq[ColumnarBatch], + dependentKernels: Seq[ExpressionEvaluator], + dependentKernelIterators: Seq[AbstractBatchIterator] + ): Iterator[ColumnarBatch] = { + ExecutorManager.tryTaskSet(numaBindingInfo) + GlutenConfig.getConf + var build_elapse: Long = 0 + var eval_elapse: Long = 0 + val execTempDir = GlutenConfig.getTempFile + val jarList = listJars.map(jarUrl => { + logWarning(s"Get Codegened library Jar ${jarUrl}") + UserAddedJarUtils.fetchJarFromSpark( + jarUrl, + execTempDir, + s"spark-columnar-plugin-codegen-precompile-${signature}.jar", + sparkConf) + s"${execTempDir}/spark-columnar-plugin-codegen-precompile-${signature}.jar" + }) + + val transKernel = new ExpressionEvaluator(jarList.toList.asJava) + val inBatchIter = new ColumnarNativeIterator(iter.asJava) + val inBatchIters = new java.util.ArrayList[AbstractColumnarNativeIterator]() + inBatchIters.add(inBatchIter) + // we need to complete dependency RDD's firstly + val beforeBuild = System.nanoTime() + val outputSchema = ArrowConverterUtils.toArrowSchema(outputAttributes) + val nativeIterator = transKernel.createKernelWithBatchIterator(rootNode, inBatchIters) + build_elapse += System.nanoTime() - beforeBuild + val resultStructType = ArrowUtils.fromArrowSchema(outputSchema) + val resIter = streamedSortPlan match { + case t: TransformSupport => + new Iterator[ColumnarBatch] { + override def hasNext: Boolean = { + val res = nativeIterator.hasNext + // if (res == false) updateMetrics(nativeIterator) + res + } + + override def next(): ColumnarBatch = { + val beforeEval = System.nanoTime() + val output_rb = nativeIterator.next.asInstanceOf[ArrowRecordBatch] + if (output_rb == null) { + eval_elapse += System.nanoTime() - beforeEval + val resultColumnVectors = + ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray + return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0) + } + val outputNumRows = output_rb.getLength + val outSchema = ArrowConverterUtils.toArrowSchema(outputAttributes) + val output = ArrowConverterUtils.fromArrowRecordBatch(outSchema, output_rb) + ArrowConverterUtils.releaseArrowRecordBatch(output_rb) + eval_elapse += System.nanoTime() - beforeEval + new ColumnarBatch(output.map(v => v.asInstanceOf[ColumnVector]), outputNumRows) + } + } + case _ => + throw new UnsupportedOperationException( + s"streamedSortPlan should support transformation") + } + var closed = false + + def close = { + closed = true + pipelineTime += (eval_elapse + build_elapse) / 1000000 + buildRelationBatchHolder.foreach(_.close) // fixing: ref cnt goes nagative + dependentKernels.foreach(_.close) + dependentKernelIterators.foreach(_.close) + nativeIterator.close() + // relationHolder.clear() + } + + SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit](_ => { + close + }) + new CloseableColumnBatchIterator(resIter) + } + + /** + * Generate columnar native iterator. + * + * @return + */ + override def genColumnarNativeIterator(delegated: Iterator[ColumnarBatch] + ): ColumnarNativeIterator = { + new ColumnarNativeIterator(delegated.asJava) + } + + /** + * Generate BatchIterator for ExpressionEvaluator. + * + * @return + */ + override def genBatchIterator(wsPlan: Array[Byte], + iterList: Seq[AbstractColumnarNativeIterator], + jniWrapper: ExpressionEvaluatorJniWrapper + ): AbstractBatchIterator = { + val memoryPool = SparkMemoryUtils.contextMemoryPool(); + val poolId = memoryPool.getNativeInstanceId(); + val batchIteratorInstance = jniWrapper.nativeCreateKernelWithIterator( + poolId, wsPlan, iterList.toArray); + new BatchIterator(batchIteratorInstance) + } + + /** + * Get the backend api name. + * + * @return + */ + override def getBackendName: String = GlutenConfig.GLUTEN_VELOX_BACKEND +} diff --git a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxSparkPlanExecApi.scala b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxSparkPlanExecApi.scala new file mode 100644 index 000000000000..9d4d05929dde --- /dev/null +++ b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxSparkPlanExecApi.scala @@ -0,0 +1,110 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.backendsapi.velox + +import scala.collection.JavaConverters._ + +import io.glutenproject.backendsapi.ISparkPlanExecApi +import io.glutenproject.GlutenConfig +import io.glutenproject.execution.{NativeColumnarToRowExec, RowToArrowColumnarExec, VeloxNativeColumnarToRowExec, VeloxRowToArrowColumnarExec} +import io.glutenproject.vectorized.ArrowColumnarBatchSerializer +import org.apache.spark.ShuffleDependency + +import org.apache.spark.rdd.RDD +import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.{GenShuffleWriterParameters, GlutenShuffleWriterWrapper} +import org.apache.spark.shuffle.utils.VeloxShuffleUtil +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.utils.VeloxExecUtil +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch + +class VeloxSparkPlanExecApi extends ISparkPlanExecApi { + + /** + * Generate NativeColumnarToRowExec. + * + * @param child + * @return + */ + override def genNativeColumnarToRowExec(child: SparkPlan): NativeColumnarToRowExec = + new VeloxNativeColumnarToRowExec(child) + + /** + * Generate RowToArrowColumnarExec. + * + * @param child + * @return + */ + override def genRowToArrowColumnarExec(child: SparkPlan): RowToArrowColumnarExec = + new VeloxRowToArrowColumnarExec(child) + + /** + * Generate ShuffleDependency for ColumnarShuffleExchangeExec. + * + * @return + */ + override def genShuffleDependency(rdd: RDD[ColumnarBatch], + outputAttributes: Seq[Attribute], + newPartitioning: Partitioning, + serializer: Serializer, + writeMetrics: Map[String, SQLMetric], + dataSize: SQLMetric, + bytesSpilled: SQLMetric, + numInputRows: SQLMetric, + computePidTime: SQLMetric, + splitTime: SQLMetric, + spillTime: SQLMetric, + compressTime: SQLMetric, + prepareTime: SQLMetric + ): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = { + VeloxExecUtil.genShuffleDependency(rdd, outputAttributes, newPartitioning, + serializer, writeMetrics, dataSize, bytesSpilled, numInputRows, + computePidTime, splitTime, spillTime, compressTime, prepareTime) + } + + /** + * Generate ColumnarShuffleWriter for ColumnarShuffleManager. + * + * @return + */ + override def genColumnarShuffleWriter[K, V](parameters: GenShuffleWriterParameters[K, V] + ): GlutenShuffleWriterWrapper[K, V] = { + VeloxShuffleUtil.genColumnarShuffleWriter(parameters) + } + + /** + * Generate ColumnarBatchSerializer for ColumnarShuffleExchangeExec. + * + * @return + */ + override def createColumnarBatchSerializer(schema: StructType, + readBatchNumRows: SQLMetric, + numOutputRows: SQLMetric): Serializer = { + new ArrowColumnarBatchSerializer(schema, readBatchNumRows, numOutputRows) + } + + /** + * Get the backend api name. + * + * @return + */ + override def getBackendName: String = GlutenConfig.GLUTEN_VELOX_BACKEND +} diff --git a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxTransformerApi.scala b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxTransformerApi.scala new file mode 100644 index 000000000000..149f572a806a --- /dev/null +++ b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxTransformerApi.scala @@ -0,0 +1,65 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.backendsapi.velox + +import scala.collection.JavaConverters._ + +import io.glutenproject.backendsapi.ITransformerApi +import io.glutenproject.GlutenConfig +import io.glutenproject.expression.ArrowConverterUtils + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning} + +class VeloxTransformerApi extends ITransformerApi with Logging { + + /** + * Do validate for ColumnarShuffleExchangeExec. + * + * @return + */ + override def validateColumnarShuffleExchangeExec(outputPartitioning: Partitioning, + outputAttributes: Seq[Attribute] + ): Boolean = { + // check input datatype + for (attr <- outputAttributes) { + try ArrowConverterUtils.createArrowField(attr) catch { + case e: UnsupportedOperationException => + logInfo(s"${attr.dataType} is not supported in VeloxColumnarShuffledExchangeExec.") + return false + } + } + outputPartitioning match { + case HashPartitioning(exprs, n) => + exprs.foreach(expr => { + if (!expr.isInstanceOf[Attribute]) { + logInfo("Expressions are not supported in HashPartitioning.") + return false + }}) + case _ => + } + true + } + + /** + * Get the backend api name. + * + * @return + */ + override def getBackendName: String = GlutenConfig.GLUTEN_VELOX_BACKEND +} diff --git a/jvm/src/main/scala/io/glutenproject/execution/BroadcastColumnarRDD.scala b/backends-velox/src/main/scala/io/glutenproject/execution/BroadcastColumnarRDD.scala similarity index 83% rename from jvm/src/main/scala/io/glutenproject/execution/BroadcastColumnarRDD.scala rename to backends-velox/src/main/scala/io/glutenproject/execution/BroadcastColumnarRDD.scala index 33866e092bef..f4106e8d560e 100644 --- a/jvm/src/main/scala/io/glutenproject/execution/BroadcastColumnarRDD.scala +++ b/backends-velox/src/main/scala/io/glutenproject/execution/BroadcastColumnarRDD.scala @@ -18,6 +18,7 @@ package io.glutenproject.execution import io.glutenproject.vectorized.CloseableColumnBatchIterator + import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.execution._ @@ -27,11 +28,11 @@ import org.apache.spark.sql.vectorized.ColumnarBatch private final case class BroadcastColumnarRDDPartition(index: Int) extends Partition case class BroadcastColumnarRDD( - @transient private val sc: SparkContext, - metrics: Map[String, SQLMetric], - numPartitioning: Int, - inputByteBuf: broadcast.Broadcast[ColumnarHashedRelation]) - extends RDD[ColumnarBatch](sc, Nil) { + @transient private val sc: SparkContext, + metrics: Map[String, SQLMetric], + numPartitioning: Int, + inputByteBuf: broadcast.Broadcast[ColumnarHashedRelation]) + extends RDD[ColumnarBatch](sc, Nil) { override protected def getPartitions: Array[Partition] = { (0 until numPartitioning).map { index => new BroadcastColumnarRDDPartition(index) }.toArray diff --git a/jvm/src/main/scala/io/glutenproject/execution/DataToArrowColumnarExec.scala b/backends-velox/src/main/scala/io/glutenproject/execution/DataToArrowColumnarExec.scala similarity index 67% rename from jvm/src/main/scala/io/glutenproject/execution/DataToArrowColumnarExec.scala rename to backends-velox/src/main/scala/io/glutenproject/execution/DataToArrowColumnarExec.scala index cbf8c6bdc510..87595d8e9b4f 100644 --- a/jvm/src/main/scala/io/glutenproject/execution/DataToArrowColumnarExec.scala +++ b/backends-velox/src/main/scala/io/glutenproject/execution/DataToArrowColumnarExec.scala @@ -17,49 +17,17 @@ package io.glutenproject.execution -import java.nio.ByteBuffer -import java.util.concurrent.TimeUnit._ +import scala.collection.JavaConverters._ -import io.glutenproject.vectorized._ -import io.glutenproject.GlutenConfig +import org.apache.spark.broadcast -import org.apache.spark.{broadcast, TaskContext} import org.apache.spark.rdd.RDD -import org.apache.spark.util.{Utils, UserAddedJarUtils} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{ - Attribute, - Expression, - SortOrder, - UnsafeProjection -} -import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder, UnsafeProjection} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} - -import scala.collection.JavaConverters._ -import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.expressions.BoundReference -import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReference -import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} -import scala.collection.mutable.ListBuffer -import org.apache.arrow.vector.ipc.message.ArrowFieldNode -import org.apache.arrow.vector.ipc.message.ArrowRecordBatch -import org.apache.arrow.vector.types.pojo.ArrowType -import org.apache.arrow.vector.types.pojo.Field -import org.apache.arrow.vector.types.pojo.Schema -import org.apache.arrow.gandiva.expression._ -import org.apache.arrow.gandiva.evaluator._ - -import org.apache.arrow.memory.ArrowBuf -import io.netty.buffer.ByteBuf -import com.google.common.collect.Lists; - -import io.glutenproject.expression._ -import io.glutenproject.vectorized.ExpressionEvaluator -import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec +import org.apache.spark.sql.vectorized.ColumnarBatch /** * Performs a hash join of two child relations by first shuffling the data using the join keys. diff --git a/backends-velox/src/main/scala/io/glutenproject/expression/ArrowConverterUtils.scala b/backends-velox/src/main/scala/io/glutenproject/expression/ArrowConverterUtils.scala new file mode 100644 index 000000000000..af75964a947d --- /dev/null +++ b/backends-velox/src/main/scala/io/glutenproject/expression/ArrowConverterUtils.scala @@ -0,0 +1,482 @@ +/* + * 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 io.glutenproject.expression + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, InputStream, IOException, OutputStream} +import java.nio.channels.Channels +import scala.collection.JavaConverters._ + +import com.google.common.collect.Lists +import io.glutenproject.vectorized.ArrowWritableColumnVector +import io.netty.buffer.{ByteBufAllocator, ByteBufOutputStream} +import org.apache.arrow.flatbuf.MessageHeader +import org.apache.arrow.gandiva.exceptions.GandivaException +import org.apache.arrow.gandiva.expression.{ExpressionTree, TreeBuilder, TreeNode} +import org.apache.arrow.gandiva.ipc.GandivaTypes +import org.apache.arrow.gandiva.ipc.GandivaTypes.ExpressionList +import org.apache.arrow.memory.BufferAllocator +import org.apache.arrow.vector.ipc.{ReadChannel, WriteChannel} +import org.apache.arrow.vector.ipc.message.{ArrowRecordBatch, IpcOption, MessageChannelReader, MessageResult, MessageSerializer} +import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema} +import org.apache.arrow.vector.ValueVector +import org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID +import org.apache.arrow.vector.types.TimeUnit + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.util.DateTimeConstants +import org.apache.spark.sql.execution.datasources.v2.arrow.{SparkSchemaUtils, SparkVectorUtils} +import org.apache.spark.sql.types.{ArrayType, BooleanType, ByteType, DataType, DateType, DecimalType, DoubleType, FloatType, IntegerType, LongType, MapType, ShortType, StringType, StructField, StructType, TimestampType} +import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} + +object ArrowConverterUtils extends Logging { + + def calcuateEstimatedSize(columnarBatch: ColumnarBatch): Long = { + SparkVectorUtils.estimateSize(columnarBatch) + } + + def createArrowRecordBatch(columnarBatch: ColumnarBatch): ArrowRecordBatch = { + SparkVectorUtils.toArrowRecordBatch(columnarBatch) + } + + def createArrowRecordBatch(numRowsInBatch: Int, cols: List[ValueVector]): ArrowRecordBatch = { + SparkVectorUtils.toArrowRecordBatch(numRowsInBatch, cols) + } + + def convertToNetty(iter: Array[ColumnarBatch], out: OutputStream): Unit = { + val channel = new WriteChannel(Channels.newChannel(out)) + var schema: Schema = null + val option = new IpcOption + + iter.foreach { columnarBatch => + val vectors = (0 until columnarBatch.numCols) + .map(i => columnarBatch.column(i).asInstanceOf[ArrowWritableColumnVector]) + .toList + try { + if (schema == null) { + schema = new Schema(vectors.map(_.getValueVector().getField).asJava) + MessageSerializer.serialize(channel, schema, option) + } + val batch = ArrowConverterUtils + .createArrowRecordBatch(columnarBatch.numRows, vectors.map(_.getValueVector)) + try { + MessageSerializer.serialize(channel, batch, option) + } finally { + batch.close() + } + } catch { + case e => + System.err.println(s"conversion failed") + e.printStackTrace() + throw e + } + } + } + + def convertToNetty(iter: Array[ColumnarBatch]): Array[Byte] = { + val innerBuf = ByteBufAllocator.DEFAULT.buffer() + val outStream = new ByteBufOutputStream(innerBuf) + convertToNetty(iter, outStream) + val bytes = new Array[Byte](innerBuf.readableBytes); + innerBuf.getBytes(innerBuf.readerIndex, bytes); + innerBuf.release() + outStream.close() + bytes + } + + def convertFromNetty( + attributes: Seq[Attribute], + input: InputStream): Iterator[ColumnarBatch] = { + new Iterator[ColumnarBatch] { + val allocator = ArrowWritableColumnVector.getOffRecordAllocator + var messageReader = + new MessageChannelReader(new ReadChannel(Channels.newChannel(input)), allocator) + var schema: Schema = null + var result: MessageResult = null + + override def hasNext: Boolean = + if (input.available > 0) { + return true + } else { + messageReader.close + return false + } + override def next(): ColumnarBatch = { + if (input.available == 0) { + if (attributes == null) { + return null + } + val resultStructType = StructType( + attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) + val resultColumnVectors = + ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray + return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0) + } + try { + if (schema == null) { + result = messageReader.readNext(); + if (result == null) { + throw new IOException("Unexpected end of input. Missing schema."); + } + if (result.getMessage().headerType() != MessageHeader.Schema) { + throw new IOException( + "Expected schema but header was " + result.getMessage().headerType()); + } + schema = MessageSerializer.deserializeSchema(result.getMessage()); + } + result = messageReader.readNext(); + if (result.getMessage().headerType() != MessageHeader.RecordBatch) { + throw new IOException( + "Expected recordbatch but header was " + result.getMessage().headerType()); + } + var bodyBuffer = result.getBodyBuffer(); + + // For zero-length batches, need an empty buffer to deserialize the batch + if (bodyBuffer == null) { + bodyBuffer = allocator.getEmpty(); + } + + val batch = MessageSerializer.deserializeRecordBatch(result.getMessage(), bodyBuffer); + val vectors = fromArrowRecordBatch(schema, batch, allocator) + val length = batch.getLength + batch.close + new ColumnarBatch(vectors.map(_.asInstanceOf[ColumnVector]), length) + } catch { + case e: Throwable => + messageReader.close + throw e + } + } + } + } + + def convertFromNetty( + attributes: Seq[Attribute], + data: Array[Array[Byte]], + columnIndices: Array[Int] = null): Iterator[ColumnarBatch] = { + if (data.size == 0) { + return new Iterator[ColumnarBatch] { + override def hasNext: Boolean = false + override def next(): ColumnarBatch = { + val resultStructType = if (columnIndices == null) { + StructType( + attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) + } else { + StructType( + columnIndices + .map(i => attributes(i)) + .map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) + } + val resultColumnVectors = + ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray + return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0) + } + } + } + new Iterator[ColumnarBatch] { + var array_id = 0 + val allocator = ArrowWritableColumnVector.getOffRecordAllocator + var input = new ByteArrayInputStream(data(array_id)) + var messageReader = + new MessageChannelReader(new ReadChannel(Channels.newChannel(input)), allocator) + var schema: Schema = null + var result: MessageResult = null + + override def hasNext: Boolean = + if (array_id < (data.size - 1) || input.available > 0) { + return true + } else { + messageReader.close + return false + } + override def next(): ColumnarBatch = { + if (input.available == 0) { + messageReader.close + array_id += 1 + input = new ByteArrayInputStream(data(array_id)) + messageReader = + new MessageChannelReader(new ReadChannel(Channels.newChannel(input)), allocator) + } + if (input.available == 0) { + val resultStructType = StructType( + attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) + val resultColumnVectors = + ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray + return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0) + } + try { + if (schema == null) { + result = messageReader.readNext(); + + if (result == null) { + throw new IOException("Unexpected end of input. Missing schema."); + } + + if (result.getMessage().headerType() != MessageHeader.Schema) { + throw new IOException( + "Expected schema but header was " + result.getMessage().headerType()); + } + + schema = MessageSerializer.deserializeSchema(result.getMessage()); + + } + + result = messageReader.readNext(); + if (result.getMessage().headerType() == MessageHeader.Schema) { + result = messageReader.readNext(); + } + + if (result.getMessage().headerType() != MessageHeader.RecordBatch) { + throw new IOException( + "Expected recordbatch but header was " + result.getMessage().headerType()); + } + var bodyBuffer = result.getBodyBuffer(); + + // For zero-length batches, need an empty buffer to deserialize the batch + if (bodyBuffer == null) { + bodyBuffer = allocator.getEmpty(); + } + + val batch = MessageSerializer.deserializeRecordBatch(result.getMessage(), bodyBuffer); + val vectors = fromArrowRecordBatch(schema, batch, allocator) + val length = batch.getLength + batch.close + if (columnIndices == null) { + new ColumnarBatch(vectors.map(_.asInstanceOf[ColumnVector]), length) + } else { + new ColumnarBatch( + columnIndices.map(i => vectors(i).asInstanceOf[ColumnVector]), + length) + } + + } catch { + case e: Throwable => + messageReader.close + throw e + } + } + } + } + + def fromArrowRecordBatch( + recordBatchSchema: Schema, + recordBatch: ArrowRecordBatch, + allocator: BufferAllocator = null): Array[ArrowWritableColumnVector] = { + val numRows = recordBatch.getLength() + ArrowWritableColumnVector.loadColumns(numRows, recordBatchSchema, recordBatch, allocator) + } + + def releaseArrowRecordBatch(recordBatch: ArrowRecordBatch): Unit = { + if (recordBatch != null) { + recordBatch.close() + } + } + + def releaseArrowRecordBatchList(recordBatchList: Array[ArrowRecordBatch]): Unit = { + recordBatchList.foreach({ recordBatch => + if (recordBatch != null) + releaseArrowRecordBatch(recordBatch) + }) + } + + def combineArrowRecordBatch(rb1: ArrowRecordBatch, rb2: ArrowRecordBatch): ArrowRecordBatch = { + val numRows = rb1.getLength() + val rb1_nodes = rb1.getNodes() + val rb2_nodes = rb2.getNodes() + val rb1_bufferlist = rb1.getBuffers() + val rb2_bufferlist = rb2.getBuffers() + + val combined_nodes = rb1_nodes.addAll(rb2_nodes) + val combined_bufferlist = rb1_bufferlist.addAll(rb2_bufferlist) + new ArrowRecordBatch(numRows, rb1_nodes, rb1_bufferlist) + } + + def toArrowSchema(attributes: Seq[Attribute]): Schema = { + val fields = attributes.map(attr => { + Field + .nullable(s"${attr.name}#${attr.exprId.id}", CodeGeneration.getResultType(attr.dataType)) + }) + new Schema(fields.toList.asJava) + } + + def toArrowSchema(schema: StructType): Schema = { + val fields = schema + .map(field => { + Field.nullable(field.name, CodeGeneration.getResultType(field.dataType)) + }) + new Schema(fields.toList.asJava) + } + + + @throws[IOException] + def getSchemaBytesBuf(schema: Schema): Array[Byte] = { + val out: ByteArrayOutputStream = new ByteArrayOutputStream + MessageSerializer.serialize(new WriteChannel(Channels.newChannel(out)), schema) + out.toByteArray + } + + @throws[IOException] + def getSchemaFromBytesBuf(schema: Array[Byte]): Schema = { + val in: ByteArrayInputStream = new ByteArrayInputStream(schema) + MessageSerializer.deserializeSchema(new ReadChannel(Channels.newChannel(in))) + } + + @throws[GandivaException] + def getExprListBytesBuf(exprs: List[ExpressionTree]): Array[Byte] = { + val builder: ExpressionList.Builder = GandivaTypes.ExpressionList.newBuilder + exprs.foreach { expr => builder.addExprs(expr.toProtobuf) } + builder.build.toByteArray + } + + def checkIfTypeSupported(dt: DataType): Unit = dt match { + case d: BooleanType => + case d: ByteType => + case d: ShortType => + case d: IntegerType => + case d: LongType => + case d: FloatType => + case d: DoubleType => + case d: StringType => + case d: DateType => + case d: DecimalType => + case d: TimestampType => + case _ => + throw new UnsupportedOperationException(s"Unsupported data type: $dt") + } + + def createArrowField(name: String, dt: DataType): Field = dt match { + case at: ArrayType => + new Field( + name, + FieldType.nullable(ArrowType.List.INSTANCE), + Lists.newArrayList(createArrowField(s"${name}_${dt}", at.elementType))) + case mt: MapType => + throw new UnsupportedOperationException(s"${dt} is not supported yet") + case st: StructType => + throw new UnsupportedOperationException(s"${dt} is not supported yet") + case _ => + Field.nullable(name, CodeGeneration.getResultType(dt)) + } + + def createArrowField(attr: Attribute): Field = + createArrowField(s"${attr.name}#${attr.exprId.id}", attr.dataType) + + private def asTimestampType(inType: ArrowType): ArrowType.Timestamp = { + if (inType.getTypeID != ArrowTypeID.Timestamp) { + throw new IllegalArgumentException(s"Value type to convert must be timestamp") + } + inType.asInstanceOf[ArrowType.Timestamp] + } + + def convertTimestampZone(inNode: TreeNode, inType: ArrowType, + toZone: String): (TreeNode, ArrowType) = { + throw new UnsupportedOperationException("not implemented") // fixme 20210602 hongze + val inTimestamp = asTimestampType(inType) + val fromZone = inTimestamp.getTimezone + + val (outNode0: TreeNode, outTimestamp0: ArrowType.Timestamp) = + if (SparkSchemaUtils.timeZoneIDEquals(fromZone, toZone)) { + val outType = new ArrowType.Timestamp(inTimestamp.getUnit, toZone) + (inNode, outType) + } else { + // todo conversion + } + (outNode0, outTimestamp0) + } + + def convertTimestampToMilli(inNode: TreeNode, inType: ArrowType): (TreeNode, ArrowType) = { + val inTimestamp = asTimestampType(inType) + inTimestamp.getUnit match { + case TimeUnit.MILLISECOND => (inNode, inType) + case TimeUnit.MICROSECOND => + // truncate from micro to milli + val outType = new ArrowType.Timestamp(TimeUnit.MILLISECOND, + inTimestamp.getTimezone) + (TreeBuilder.makeFunction( + "convertTimestampUnit", + Lists.newArrayList(inNode), outType), outType) + } + } + + def convertTimestampToMicro(inNode: TreeNode, inType: ArrowType): (TreeNode, ArrowType) = { + val inTimestamp = asTimestampType(inType) + inTimestamp.getUnit match { + case TimeUnit.MICROSECOND => (inNode, inType) + case TimeUnit.MILLISECOND => + // truncate from micro to milli + val outType = new ArrowType.Timestamp(TimeUnit.MICROSECOND, + inTimestamp.getTimezone) + (TreeBuilder.makeFunction( + "convertTimestampUnit", + Lists.newArrayList(inNode), outType), outType) + } + } + + def toInt32(inNode: TreeNode, inType: ArrowType): (TreeNode, ArrowType) = { + val toType = ArrowUtils.toArrowType(IntegerType, null) + val toNode = TreeBuilder.makeFunction("castINT", Lists.newArrayList(inNode), + toType) + (toNode, toType) + } + + // use this carefully + def toGandivaMicroUTCTimestamp(inNode: TreeNode, inType: ArrowType, + timeZoneId: Option[String] = None): (TreeNode, ArrowType) = { + val zoneId = timeZoneId.orElse(Some(SparkSchemaUtils.getLocalTimezoneID())).get + val utcTimestampNodeOriginal = inNode + val inTimestampType = asTimestampType(inType) + val inTimestampTypeUTC = new ArrowType.Timestamp(inTimestampType.getUnit, + "UTC") + ArrowConverterUtils.convertTimestampToMicro(utcTimestampNodeOriginal, + inTimestampTypeUTC) + } + + // use this carefully + def toGandivaTimestamp(inNode: TreeNode, inType: ArrowType, + timeZoneId: Option[String] = None): (TreeNode, ArrowType) = { + val zoneId = timeZoneId.orElse(Some(SparkSchemaUtils.getLocalTimezoneID())).get + + val utcTimestampNodeOriginal = inNode + val utcTimestampNodeMilli = ArrowConverterUtils.convertTimestampToMilli( + utcTimestampNodeOriginal,inType)._1 + val utcTimestampNodeLong = TreeBuilder.makeFunction("castBIGINT", + Lists.newArrayList(utcTimestampNodeMilli), new ArrowType.Int(64, + true)) + val diff = SparkSchemaUtils.getTimeZoneIDOffset(zoneId) * + DateTimeConstants.MILLIS_PER_SECOND + + val localizedTimestampNodeLong = TreeBuilder.makeFunction("add", + Lists.newArrayList(utcTimestampNodeLong, + TreeBuilder.makeLiteral(java.lang.Long.valueOf(diff))), + new ArrowType.Int(64, true)) + val localized = new ArrowType.Timestamp(TimeUnit.MILLISECOND, null) + val localizedTimestampNode = TreeBuilder.makeFunction("castTIMESTAMP", + Lists.newArrayList(localizedTimestampNodeLong), localized) + (localizedTimestampNode, localized) + } + + def toSparkTimestamp(inNode: TreeNode, inType: ArrowType, + timeZoneId: Option[String] = None): (TreeNode, ArrowType) = { + throw new UnsupportedOperationException() + } + + override def toString(): String = { + s"ArrowConverterUtils" + } +} diff --git a/jvm/src/main/scala/io/glutenproject/expression/CodeGeneration.scala b/backends-velox/src/main/scala/io/glutenproject/expression/CodeGeneration.scala similarity index 100% rename from jvm/src/main/scala/io/glutenproject/expression/CodeGeneration.scala rename to backends-velox/src/main/scala/io/glutenproject/expression/CodeGeneration.scala diff --git a/backends-velox/src/main/scala/io/glutenproject/utils/VeloxImplicitClass.scala b/backends-velox/src/main/scala/io/glutenproject/utils/VeloxImplicitClass.scala new file mode 100644 index 000000000000..510bf0d42862 --- /dev/null +++ b/backends-velox/src/main/scala/io/glutenproject/utils/VeloxImplicitClass.scala @@ -0,0 +1,43 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.utils + +import io.glutenproject.vectorized.ArrowWritableColumnVector +import org.apache.arrow.vector.util.VectorBatchAppender + +import org.apache.spark.sql.vectorized.ColumnarBatch + +object VeloxImplicitClass { + + implicit class ArrowColumnarBatchRetainer(val cb: ColumnarBatch) { + def retain(): Unit = { + (0 until cb.numCols).toList.foreach(i => + cb.column(i).asInstanceOf[ArrowWritableColumnVector].retain()) + } + } + + def coalesce(targetBatch: ColumnarBatch, batchesToAppend: List[ColumnarBatch]): Unit = { + (0 until targetBatch.numCols).toList.foreach { i => + val targetVector = + targetBatch.column(i).asInstanceOf[ArrowWritableColumnVector].getValueVector + val vectorsToAppend = batchesToAppend.map { cb => + cb.column(i).asInstanceOf[ArrowWritableColumnVector].getValueVector + } + VectorBatchAppender.batchAppend(targetVector, vectorsToAppend: _*) + } + } +} diff --git a/backends-velox/src/main/scala/io/glutenproject/utils/VeloxTransformerUtil.scala b/backends-velox/src/main/scala/io/glutenproject/utils/VeloxTransformerUtil.scala new file mode 100644 index 000000000000..3859561a14f9 --- /dev/null +++ b/backends-velox/src/main/scala/io/glutenproject/utils/VeloxTransformerUtil.scala @@ -0,0 +1,71 @@ +/* + * 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 io.glutenproject.utils + +import scala.collection.JavaConverters._ +import com.google.common.collect.Lists + +import io.glutenproject.expression.CodeGeneration +import org.apache.arrow.gandiva.expression.{TreeBuilder, TreeNode} +import org.apache.arrow.vector.types.pojo.{ArrowType, Field} + +import org.apache.spark.sql.catalyst.expressions.Attribute + +object VeloxTransformerUtil { + + def prepareRelationFunction( + keyAttributes: Seq[Attribute], + outputAttributes: Seq[Attribute]): TreeNode = { + val outputFieldList: List[Field] = outputAttributes.toList.map(attr => { + Field + .nullable(s"${attr.name.toUpperCase()}#${attr.exprId.id}", + CodeGeneration.getResultType(attr.dataType)) + }) + + val keyFieldList: List[Field] = keyAttributes.toList.map(attr => { + val field = Field + .nullable(s"${attr.name.toUpperCase()}#${attr.exprId.id}", + CodeGeneration.getResultType(attr.dataType)) + if (outputFieldList.indexOf(field) == -1) { + throw new UnsupportedOperationException(s"CachedRelation not found" + + s"${attr.name.toUpperCase()}#${attr.exprId.id} in ${outputAttributes}") + } + field + }) + + val key_args_node = TreeBuilder.makeFunction( + "key_field", + keyFieldList + .map(field => { + TreeBuilder.makeField(field) + }) + .asJava, + new ArrowType.Int(32, true) /*dummy ret type, won't be used*/ ) + + val cachedRelationFuncName = "CachedRelation" + val cached_relation_func = TreeBuilder.makeFunction( + cachedRelationFuncName, + Lists.newArrayList(key_args_node), + new ArrowType.Int(32, true) /*dummy ret type, won't be used*/ ) + + TreeBuilder.makeFunction( + "standalone", + Lists.newArrayList(cached_relation_func), + new ArrowType.Int(32, true)) + } +} diff --git a/jvm/src/main/scala/io/glutenproject/vectorized/ArrowColumnarBatchSerializer.scala b/backends-velox/src/main/scala/io/glutenproject/vectorized/ArrowColumnarBatchSerializer.scala similarity index 91% rename from jvm/src/main/scala/io/glutenproject/vectorized/ArrowColumnarBatchSerializer.scala rename to backends-velox/src/main/scala/io/glutenproject/vectorized/ArrowColumnarBatchSerializer.scala index 92575c6e708f..ab0dd7a37643 100644 --- a/jvm/src/main/scala/io/glutenproject/vectorized/ArrowColumnarBatchSerializer.scala +++ b/backends-velox/src/main/scala/io/glutenproject/vectorized/ArrowColumnarBatchSerializer.scala @@ -26,29 +26,22 @@ import scala.collection.mutable.ListBuffer import scala.reflect.ClassTag import io.glutenproject.GlutenConfig -import io.glutenproject.expression.ConverterUtils +import io.glutenproject.expression.ArrowConverterUtils import org.apache.arrow.dataset.jni.UnsafeRecordBatchSerializer -import org.apache.arrow.memory.ArrowBuf -import org.apache.arrow.memory.BufferAllocator +import org.apache.arrow.memory.{ArrowBuf, BufferAllocator} +import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot} import org.apache.arrow.vector.ipc.ArrowStreamReader -import org.apache.arrow.vector.VectorLoader -import org.apache.arrow.vector.VectorSchemaRoot import org.apache.arrow.vector.types.pojo.Schema - import org.apache.spark.SparkEnv + import org.apache.spark.internal.Logging -import org.apache.spark.serializer.DeserializationStream -import org.apache.spark.serializer.SerializationStream -import org.apache.spark.serializer.Serializer -import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkVectorUtils +import org.apache.spark.serializer.{DeserializationStream, SerializationStream, Serializer, SerializerInstance} +import org.apache.spark.sql.execution.datasources.v2.arrow.{SparkMemoryUtils, SparkVectorUtils} import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.ArrowUtils -import org.apache.spark.sql.vectorized.ColumnVector -import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} class ArrowColumnarBatchSerializer( schema: StructType, readBatchNumRows: SQLMetric, numOutputRows: SQLMetric) @@ -198,7 +191,7 @@ private class ArrowColumnarBatchSerializerInstance( private def decompressVectors(): Unit = { if (jniWrapper == null) { jniWrapper = new ShuffleDecompressionJniWrapper - schemaHolderId = jniWrapper.make(ConverterUtils.getSchemaBytesBuf(root.getSchema)) + schemaHolderId = jniWrapper.make(ArrowConverterUtils.getSchemaBytesBuf(root.getSchema)) } if (vectorLoader == null) { vectorLoader = new VectorLoader(root) diff --git a/jvm/src/main/scala/io/glutenproject/vectorized/CloseableColumnBatchIterator.scala b/backends-velox/src/main/scala/io/glutenproject/vectorized/CloseableColumnBatchIterator.scala similarity index 92% rename from jvm/src/main/scala/io/glutenproject/vectorized/CloseableColumnBatchIterator.scala rename to backends-velox/src/main/scala/io/glutenproject/vectorized/CloseableColumnBatchIterator.scala index 23575c978d90..4068a8fb901e 100644 --- a/jvm/src/main/scala/io/glutenproject/vectorized/CloseableColumnBatchIterator.scala +++ b/backends-velox/src/main/scala/io/glutenproject/vectorized/CloseableColumnBatchIterator.scala @@ -17,17 +17,18 @@ package io.glutenproject.vectorized -import org.apache.spark.internal.Logging -import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.TaskContext + +import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils +import org.apache.spark.sql.vectorized.ColumnarBatch /** * An Iterator that insures that the batches [[ColumnarBatch]]s it iterates over are all closed * properly. */ class CloseableColumnBatchIterator(itr: Iterator[ColumnarBatch]) - extends Iterator[ColumnarBatch] + extends Iterator[ColumnarBatch] with Logging { var cb: ColumnarBatch = null @@ -40,8 +41,8 @@ class CloseableColumnBatchIterator(itr: Iterator[ColumnarBatch]) } SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit]((tc: TaskContext) => { - closeCurrentBatch() - }) + closeCurrentBatch() + }) override def hasNext: Boolean = { itr.hasNext diff --git a/jvm/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala b/backends-velox/src/main/scala/org/apache/spark/shuffle/VeloxColumnarShuffleWriter.scala similarity index 71% rename from jvm/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala rename to backends-velox/src/main/scala/org/apache/spark/shuffle/VeloxColumnarShuffleWriter.scala index 4f214fd78fa2..7c7f0fe65ecc 100644 --- a/jvm/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala +++ b/backends-velox/src/main/scala/org/apache/spark/shuffle/VeloxColumnarShuffleWriter.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.{ArrayBuffer, ListBuffer} import com.google.common.annotations.VisibleForTesting import io.glutenproject.GlutenConfig -import io.glutenproject.expression.ConverterUtils +import io.glutenproject.expression.ArrowConverterUtils import io.glutenproject.spark.sql.execution.datasources.v2.arrow.Spiller import io.glutenproject.vectorized._ import org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID @@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils -class ColumnarShuffleWriter[K, V]( +class VeloxColumnarShuffleWriter[K, V]( shuffleBlockResolver: IndexShuffleBlockResolver, handle: BaseShuffleHandle[K, V, V], mapId: Long, @@ -80,7 +80,7 @@ class ColumnarShuffleWriter[K, V]( private val writeSchema = GlutenConfig.getConf.columnarShuffleWriteSchema - private val jniWrapper = ColumnarFactory.createShuffleSplitterJniWrapper(); + private val jniWrapper = new ShuffleSplitterJniWrapper private var nativeSplitter: Long = 0 @@ -139,7 +139,7 @@ class ColumnarShuffleWriter[K, V]( val startTimeForPrepare = System.nanoTime() val bufAddrs = new ListBuffer[Long]() val bufSizes = new ListBuffer[Long]() - val recordBatch = ConverterUtils.createArrowRecordBatch(cb) + val recordBatch = ArrowConverterUtils.createArrowRecordBatch(cb) recordBatch.getBuffers().asScala.foreach { buffer => bufAddrs += buffer.memoryAddress() } recordBatch.getBuffersLayout().asScala.foreach { bufLayout => bufSizes += bufLayout.getSize() @@ -149,8 +149,8 @@ class ColumnarShuffleWriter[K, V]( val startTime = System.nanoTime() val existingIntType: Boolean = if (firstRecordBatch) { // Check whether the recordbatch contain the Int data type. - val arrowSchema = ConverterUtils.getSchemaFromBytesBuf(dep.nativePartitioning.getSchema) - import scala.collection.JavaConverters._ + val arrowSchema = ArrowConverterUtils.getSchemaFromBytesBuf( + dep.nativePartitioning.getSchema) arrowSchema.getFields.asScala.find(_.getType.getTypeID == ArrowTypeID.Int).nonEmpty } else false @@ -184,7 +184,7 @@ class ColumnarShuffleWriter[K, V]( dep.splitTime.add(System.nanoTime() - startTime) dep.numInputRows.add(cb.numRows) writeMetrics.incRecordsWritten(1) - ConverterUtils.releaseArrowRecordBatch(recordBatch) + ArrowConverterUtils.releaseArrowRecordBatch(recordBatch) } } @@ -222,95 +222,15 @@ class ColumnarShuffleWriter[K, V]( mapStatus = MapStatus(blockManager.shuffleServerId, unionPartitionLengths.toArray, mapId) } - def internalCHWrite(records: Iterator[Product2[K, V]]): Unit = { - val splitterJniWrapper : CHShuffleSplitterJniWrapper = - jniWrapper.asInstanceOf[CHShuffleSplitterJniWrapper] - if (!records.hasNext) { - partitionLengths = new Array[Long](dep.partitioner.numPartitions) - shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, null) - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapId) - return - } - val dataTmp = Utils.tempFileWith(shuffleBlockResolver.getDataFile(dep.shuffleId, mapId)) - if (nativeSplitter == 0) { - nativeSplitter = splitterJniWrapper.make( - dep.nativePartitioning, - mapId, - nativeBufferSize, - defaultCompressionCodec, - dataTmp.getAbsolutePath, - localDirs) - } - while (records.hasNext) { - val cb = records.next()._2.asInstanceOf[ColumnarBatch] - if (cb.numRows == 0 || cb.numCols == 0) { - logInfo(s"Skip ColumnarBatch of ${cb.numRows} rows, ${cb.numCols} cols") - } else { - val startTimeForPrepare = System.nanoTime() - - val startTime = System.nanoTime() - firstRecordBatch = false - dep.prepareTime.add(System.nanoTime() - startTimeForPrepare) - val col = cb.column(0).asInstanceOf[CHColumnVector] - val block = col.getBlockAddress - splitterJniWrapper - .split(nativeSplitter, cb.numRows, block) - dep.splitTime.add(System.nanoTime() - startTime) - dep.numInputRows.add(cb.numRows) - writeMetrics.incRecordsWritten(1) - } - } - val startTime = System.nanoTime() - splitResult = splitterJniWrapper.stop(nativeSplitter) - - dep.splitTime.add(System.nanoTime() - startTime - splitResult.getTotalSpillTime - - splitResult.getTotalWriteTime - splitResult.getTotalComputePidTime - - splitResult.getTotalCompressTime) - dep.spillTime.add(splitResult.getTotalSpillTime) - dep.compressTime.add(splitResult.getTotalCompressTime) - dep.computePidTime.add(splitResult.getTotalComputePidTime) - dep.bytesSpilled.add(splitResult.getTotalBytesSpilled) - writeMetrics.incBytesWritten(splitResult.getTotalBytesWritten) - writeMetrics.incWriteTime(splitResult.getTotalWriteTime + splitResult.getTotalSpillTime) - - partitionLengths = splitResult.getPartitionLengths - rawPartitionLengths = splitResult.getRawPartitionLengths - try { - shuffleBlockResolver.writeIndexFileAndCommit( - dep.shuffleId, - mapId, - partitionLengths, - dataTmp) - } finally { - if (dataTmp.exists() && !dataTmp.delete()) { - logError(s"Error while deleting temp file ${dataTmp.getAbsolutePath}") - } - } - - // fixme workaround: to store uncompressed sizes on the rhs of (maybe) compressed sizes - val unionPartitionLengths = ArrayBuffer[Long]() - unionPartitionLengths ++= partitionLengths - unionPartitionLengths ++= rawPartitionLengths - mapStatus = MapStatus(blockManager.shuffleServerId, unionPartitionLengths.toArray, mapId) - } - @throws[IOException] override def write(records: Iterator[Product2[K, V]]): Unit = { - if (GlutenConfig.getConf.isClickHouseBackend) { - internalCHWrite(records) - } else { - internalWrite(records) - } + internalWrite(records) } def closeSplitter(): Unit = { jniWrapper.asInstanceOf[ShuffleSplitterJniWrapper].close(nativeSplitter) } - def closeCHSplitter(): Unit = { - jniWrapper.asInstanceOf[CHShuffleSplitterJniWrapper].close(nativeSplitter) - } - override def stop(success: Boolean): Option[MapStatus] = { try { if (stopping) { @@ -324,11 +244,7 @@ class ColumnarShuffleWriter[K, V]( } } finally { if (nativeSplitter != 0) { - if (!GlutenConfig.getConf.isClickHouseBackend) { - closeSplitter() - } else { - closeCHSplitter() - } + closeSplitter() nativeSplitter = 0 } } diff --git a/backends-velox/src/main/scala/org/apache/spark/shuffle/utils/VeloxShuffleUtil.scala b/backends-velox/src/main/scala/org/apache/spark/shuffle/utils/VeloxShuffleUtil.scala new file mode 100644 index 000000000000..bee1fdfce075 --- /dev/null +++ b/backends-velox/src/main/scala/org/apache/spark/shuffle/utils/VeloxShuffleUtil.scala @@ -0,0 +1,31 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle.utils + +import org.apache.spark.shuffle.{GenShuffleWriterParameters, GlutenShuffleWriterWrapper, VeloxColumnarShuffleWriter} + +object VeloxShuffleUtil { + + def genColumnarShuffleWriter[K, V](parameters: GenShuffleWriterParameters[K, V] + ): GlutenShuffleWriterWrapper[K, V] = { + GlutenShuffleWriterWrapper(new VeloxColumnarShuffleWriter[K, V]( + parameters.shuffleBlockResolver, + parameters.columnarShuffleHandle, + parameters.mapId, + parameters.metrics)) + } +} diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarHashedRelation.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarHashedRelation.scala similarity index 91% rename from jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarHashedRelation.scala rename to backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarHashedRelation.scala index 25a614b71724..9e65de295889 100644 --- a/jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarHashedRelation.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarHashedRelation.scala @@ -19,14 +19,15 @@ package org.apache.spark.sql.execution import java.io._ -import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Kryo, KryoSerializable} -import io.glutenproject.expression.ConverterUtils +import com.esotericsoftware.kryo.io.{Input, Output} +import io.glutenproject.expression.ArrowConverterUtils import io.glutenproject.vectorized.{ArrowWritableColumnVector, SerializableObject} +import sun.misc.Cleaner + import org.apache.spark.sql.execution.ColumnarHashedRelation.Deallocator import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.KnownSizeEstimation -import sun.misc.Cleaner class ColumnarHashedRelation( var hashRelationObj: SerializableObject, @@ -60,13 +61,13 @@ class ColumnarHashedRelation( override def writeExternal(out: ObjectOutput): Unit = { out.writeObject(hashRelationObj) - val rawArrowData = ConverterUtils.convertToNetty(arrowColumnarBatch) + val rawArrowData = ArrowConverterUtils.convertToNetty(arrowColumnarBatch) out.writeObject(rawArrowData) } override def write(kryo: Kryo, out: Output): Unit = { kryo.writeObject(out, hashRelationObj) - val rawArrowData = ConverterUtils.convertToNetty(arrowColumnarBatch) + val rawArrowData = ArrowConverterUtils.convertToNetty(arrowColumnarBatch) kryo.writeObject(out, rawArrowData) } @@ -75,7 +76,7 @@ class ColumnarHashedRelation( val rawArrowData = in.readObject().asInstanceOf[Array[Byte]] arrowColumnarBatchSize = rawArrowData.length arrowColumnarBatch = - ConverterUtils.convertFromNetty(null, new ByteArrayInputStream(rawArrowData)).toArray + ArrowConverterUtils.convertFromNetty(null, new ByteArrayInputStream(rawArrowData)).toArray createCleaner(hashRelationObj, arrowColumnarBatch) // retain all cols /*arrowColumnarBatch.foreach(cb => { @@ -90,7 +91,7 @@ class ColumnarHashedRelation( val rawArrowData = kryo.readObject(in, classOf[Array[Byte]]).asInstanceOf[Array[Byte]] arrowColumnarBatchSize = rawArrowData.length arrowColumnarBatch = - ConverterUtils.convertFromNetty(null, new ByteArrayInputStream(rawArrowData)).toArray + ArrowConverterUtils.convertFromNetty(null, new ByteArrayInputStream(rawArrowData)).toArray createCleaner(hashRelationObj, arrowColumnarBatch) // retain all cols /*arrowColumnarBatch.foreach(cb => { diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarInMemoryRelation.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarInMemoryRelation.scala similarity index 85% rename from jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarInMemoryRelation.scala rename to backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarInMemoryRelation.scala index e28124c658d7..2ce1f17435f3 100644 --- a/jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarInMemoryRelation.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarInMemoryRelation.scala @@ -18,39 +18,26 @@ package org.apache.spark.sql.execution import java.io._ -import org.apache.commons.lang3.StringUtils +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import com.esotericsoftware.kryo.io.{Input, Output} +import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.expression._ import io.glutenproject.vectorized.ArrowWritableColumnVector -import io.glutenproject.vectorized.CloseableColumnBatchIterator -import org.apache.arrow.memory.ArrowBuf -import org.apache.spark.TaskContext -import org.apache.spark.network.util.JavaUtils + import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan} -import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics} -import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.columnar.{ - CachedBatch, - CachedBatchSerializer, - SimpleMetricsCachedBatch, - SimpleMetricsCachedBatchSerializer -} +import org.apache.spark.sql.columnar.{CachedBatch, SimpleMetricsCachedBatch} import org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.vectorized.{WritableColumnVector} -import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{LongAccumulator, Utils, KnownSizeEstimation} -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer -import com.esotericsoftware.kryo.io.{Input, Output} -import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import org.apache.spark.util.KnownSizeEstimation /** * The default implementation of CachedBatch. @@ -77,20 +64,20 @@ case class ArrowCachedBatch( override def estimatedSize: Long = { var size: Long = 0 buffer.foreach(batch => { - size += ConverterUtils.calcuateEstimatedSize(batch) + size += ArrowConverterUtils.calcuateEstimatedSize(batch) }) size } override def sizeInBytes: Long = estimatedSize override def writeExternal(out: ObjectOutput): Unit = { out.writeObject(numRows) - val rawArrowData = ConverterUtils.convertToNetty(buffer) + val rawArrowData = ArrowConverterUtils.convertToNetty(buffer) out.writeObject(rawArrowData) buffer.foreach(_.close) } override def write(kryo: Kryo, out: Output): Unit = { kryo.writeObject(out, numRows) - val rawArrowData = ConverterUtils.convertToNetty(buffer) + val rawArrowData = ArrowConverterUtils.convertToNetty(buffer) kryo.writeObject(out, rawArrowData) System.out.println("ArrowCachedBatch close when write to disk") buffer.foreach(_.close) @@ -98,12 +85,12 @@ case class ArrowCachedBatch( override def readExternal(in: ObjectInput): Unit = { numRows = in.readObject().asInstanceOf[Integer] val rawArrowData = in.readObject().asInstanceOf[Array[Byte]] - buffer = ConverterUtils.convertFromNetty(null, new ByteArrayInputStream(rawArrowData)).toArray + buffer = ArrowConverterUtils.convertFromNetty(null, new ByteArrayInputStream(rawArrowData)).toArray } override def read(kryo: Kryo, in: Input): Unit = { numRows = kryo.readObject(in, classOf[Integer]).asInstanceOf[Integer] val rawArrowData = kryo.readObject(in, classOf[Array[Byte]]).asInstanceOf[Array[Byte]] - buffer = ConverterUtils.convertFromNetty(null, new ByteArrayInputStream(rawArrowData)).toArray + buffer = ArrowConverterUtils.convertFromNetty(null, new ByteArrayInputStream(rawArrowData)).toArray } } @@ -211,7 +198,7 @@ class ArrowColumnarCachedBatchSerializer extends DefaultCachedBatchSerializer { } def hasNext: Boolean = iter.hasNext } - new CloseableColumnBatchIterator(res) + BackendsApiManager.getIteratorApiInstance.genCloseableColumnBatchIterator(res) } input.mapPartitions(createAndDecompressColumn) } else { diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala similarity index 95% rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala rename to backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala index 2bec6536b987..0d57ed8f53cb 100644 --- a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala @@ -17,25 +17,18 @@ package org.apache.spark.sql.execution.datasources.v2.arrow -import io.glutenproject.GlutenConfig - import java.io.PrintWriter -import java.util import java.util.UUID import scala.collection.JavaConverters._ -import io.glutenproject.spark.sql.execution.datasources.v2.arrow._ import com.sun.xml.internal.messaging.saaj.util.ByteOutputStream +import io.glutenproject.spark.sql.execution.datasources.v2.arrow._ +import java.util 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.arrow.memory.{AllocationListener, BufferAllocator, MemoryChunkCleaner, MemoryChunkManager, RootAllocator} +import org.apache.spark.{SparkEnv, TaskContext} + import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.memory.TaskMemoryManager @@ -111,9 +104,7 @@ object SparkMemoryUtils extends Logging { .newChildAllocator("CHILD-ALLOC-BUFFER-IMPORT", allocListenerForBufferImport, 0L, Long.MaxValue) - val defaultMemoryPool: NativeMemoryPoolWrapper = if (GlutenConfig.getConf.isClickHouseBackend) { - null - } else { + val defaultMemoryPool: NativeMemoryPoolWrapper = { val rl = new SparkManagedReservationListener( new NativeSQLMemoryConsumer(getTaskMemoryManager(), Spiller.NO_OP), sharedMetrics) diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkSchemaUtils.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkSchemaUtils.scala similarity index 100% rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkSchemaUtils.scala rename to backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkSchemaUtils.scala diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkVectorUtils.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkVectorUtils.scala similarity index 100% rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkVectorUtils.scala rename to backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkVectorUtils.scala diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExecTransformer.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExecTransformer.scala similarity index 86% rename from jvm/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExecTransformer.scala rename to backends-velox/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExecTransformer.scala index c26a033385d9..a4d6e633b6e9 100644 --- a/jvm/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExecTransformer.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExecTransformer.scala @@ -17,29 +17,18 @@ package org.apache.spark.sql.execution.python -import java.io._ -import java.net._ -import java.util.concurrent.atomic.AtomicBoolean - +import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.execution.{TransformContext, TransformSupport} -import io.glutenproject.expression._ import io.glutenproject.substrait.SubstraitContext -import io.glutenproject.vectorized._ - -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer -import org.apache.arrow.vector.{ValueVector, VectorLoader, VectorSchemaRoot} -import org.apache.spark.{ContextAwareIterator, SparkEnv} import org.apache.spark.TaskContext + import org.apache.spark.api.python.ChainedPythonFunctions import org.apache.spark.sql.catalyst.InternalRow 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.internal.SQLConf -import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnVector, ColumnarBatch} -import org.apache.spark.sql.types.{DataType, StructField, StructType} +import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.rdd.RDD @@ -77,7 +66,8 @@ case class ArrowEvalPythonExecTransformer(udfs: Seq[PythonUDF], resultAttrs: Seq val outputTypes = output.drop(child.output.length).map(_.dataType) // Use Coalecse to improve performance in future - val batchIter = new CloseableColumnBatchIterator(iter) + val batchIter = + BackendsApiManager.getIteratorApiInstance.genCloseableColumnBatchIterator(iter) val columnarBatchIter = new ColumnarArrowPythonRunner( funcs, diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/python/ColumnarArrowPythonRunner.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/python/ColumnarArrowPythonRunner.scala similarity index 94% rename from jvm/src/main/scala/org/apache/spark/sql/execution/python/ColumnarArrowPythonRunner.scala rename to backends-velox/src/main/scala/org/apache/spark/sql/execution/python/ColumnarArrowPythonRunner.scala index a91371f07e22..e6e48c343a9a 100644 --- a/jvm/src/main/scala/org/apache/spark/sql/execution/python/ColumnarArrowPythonRunner.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/python/ColumnarArrowPythonRunner.scala @@ -23,19 +23,17 @@ import java.util.concurrent.atomic.AtomicBoolean import io.glutenproject.expression._ import io.glutenproject.vectorized._ - -import org.apache.arrow.vector.{ValueVector, VectorLoader, VectorSchemaRoot} +import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot} import org.apache.arrow.vector.ipc.{ArrowStreamReader, ArrowStreamWriter} -import org.apache.spark.SparkEnv -import org.apache.spark.TaskContext +import org.apache.spark.{SparkEnv, TaskContext} + import org.apache.spark.api.python.{BasePythonRunner, ChainedPythonFunctions, PythonRDD, SpecialLengths} //import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution.python.PythonUDFRunner import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.util.Utils /** @@ -159,10 +157,10 @@ class ColumnarArrowPythonRunner( while (inputIterator.hasNext) { val nextBatch = inputIterator.next() numRows += nextBatch.numRows - val next_rb = ConverterUtils.createArrowRecordBatch(nextBatch) + val next_rb = ArrowConverterUtils.createArrowRecordBatch(nextBatch) loader.load(next_rb) writer.writeBatch() - ConverterUtils.releaseArrowRecordBatch(next_rb) + ArrowConverterUtils.releaseArrowRecordBatch(next_rb) } // end writes footer to the output stream and doesn't clean any resources. // It could throw exception if the output stream is closed, so it should be diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/VeloxExecUtil.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/VeloxExecUtil.scala new file mode 100644 index 000000000000..81e8e030c94a --- /dev/null +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/VeloxExecUtil.scala @@ -0,0 +1,241 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.utils + +import scala.collection.JavaConverters._ + +import io.glutenproject.expression.{ArrowConverterUtils, ExpressionConverter, ExpressionTransformer} +import io.glutenproject.substrait.expression.ExpressionNode +import io.glutenproject.substrait.rel.RelBuilder +import io.glutenproject.vectorized.{ArrowWritableColumnVector, NativePartitioning} +import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema} +import org.apache.spark.{Partitioner, RangePartitioner, ShuffleDependency} + +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.ColumnarShuffleDependency +import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, UnsafeProjection} +import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils +import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.PartitionIdPassthrough +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.MutablePair + +object VeloxExecUtil { + + def genShuffleDependency(rdd: RDD[ColumnarBatch], + outputAttributes: Seq[Attribute], + newPartitioning: Partitioning, + serializer: Serializer, + writeMetrics: Map[String, SQLMetric], + dataSize: SQLMetric, + bytesSpilled: SQLMetric, + numInputRows: SQLMetric, + computePidTime: SQLMetric, + splitTime: SQLMetric, + spillTime: SQLMetric, + compressTime: SQLMetric, + prepareTime: SQLMetric + ): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = { + val arrowFields = outputAttributes.map(attr => + ArrowConverterUtils.createArrowField(attr) + ) + def serializeSchema(fields: Seq[Field]): Array[Byte] = { + val schema = new Schema(fields.asJava) + ArrowConverterUtils.getSchemaBytesBuf(schema) + } + + // only used for fallback range partitioning + val rangePartitioner: Option[Partitioner] = newPartitioning match { + case RangePartitioning(sortingExpressions, numPartitions) => + // Extract only fields used for sorting to avoid collecting large fields that does not + // affect sorting result when deciding partition bounds in RangePartitioner + val rddForSampling = rdd.mapPartitionsInternal { iter => + // Internally, RangePartitioner runs a job on the RDD that samples keys to compute + // partition bounds. To get accurate samples, we need to copy the mutable keys. + iter.flatMap(batch => { + val rows = batch.rowIterator.asScala + val projection = + UnsafeProjection.create(sortingExpressions.map(_.child), outputAttributes) + val mutablePair = new MutablePair[InternalRow, Null]() + rows.map(row => mutablePair.update(projection(row).copy(), null)) + }) + } + // Construct ordering on extracted sort key. + val orderingAttributes = sortingExpressions.zipWithIndex.map { + case (ord, i) => + ord.copy(child = BoundReference(i, ord.dataType, ord.nullable)) + } + implicit val ordering = new LazilyGeneratedOrdering(orderingAttributes) + val part = new RangePartitioner( + numPartitions, + rddForSampling, + ascending = true, + samplePointsPerPartitionHint = SQLConf.get.rangeExchangeSampleSizePerPartition) + Some(part) + case _ => None + } + + // only used for fallback range partitioning + def computeAndAddPartitionId(cbIter: Iterator[ColumnarBatch], + partitionKeyExtractor: InternalRow => Any + ): CloseablePairedColumnarBatchIterator = { + CloseablePairedColumnarBatchIterator { + cbIter + .filter(cb => cb.numRows != 0 && cb.numCols != 0) + .map { cb => + val startTime = System.nanoTime() + val pidVec = ArrowWritableColumnVector + .allocateColumns(cb.numRows, new StructType().add("pid", IntegerType)) + .head + (0 until cb.numRows).foreach { i => + val row = cb.getRow(i) + val pid = rangePartitioner.get.getPartition(partitionKeyExtractor(row)) + pidVec.putInt(i, pid) + } + + val newColumns = (pidVec +: (0 until cb.numCols).map(cb.column)).toArray + newColumns.foreach( + _.asInstanceOf[ArrowWritableColumnVector].getValueVector.setValueCount(cb.numRows)) + computePidTime.add(System.nanoTime() - startTime) + (0, new ColumnarBatch(newColumns, cb.numRows)) + } + } + } + + val nativePartitioning: NativePartitioning = newPartitioning match { + case SinglePartition => new NativePartitioning("single", 1, serializeSchema(arrowFields)) + case RoundRobinPartitioning(n) => + new NativePartitioning("rr", n, serializeSchema(arrowFields)) + case HashPartitioning(exprs, n) => + // Function map is not expected to be used. + val functionMap = new java.util.HashMap[String, java.lang.Long]() + val exprNodeList = new java.util.ArrayList[ExpressionNode]() + exprs.foreach(expr => { + exprNodeList.add(ExpressionConverter + .replaceWithExpressionTransformer(expr, outputAttributes) + .asInstanceOf[ExpressionTransformer] + .doTransform(functionMap)) + }) + val projectRel = RelBuilder.makeProjectRel(null, exprNodeList) + new NativePartitioning( + "hash", + n, + serializeSchema(arrowFields), + projectRel.toProtobuf().toByteArray) + // range partitioning fall back to row-based partition id computation + case RangePartitioning(orders, n) => + val pidField = Field.nullable("pid", new ArrowType.Int(32, true)) + new NativePartitioning("range", n, serializeSchema(pidField +: arrowFields)) + } + + val isRoundRobin = newPartitioning.isInstanceOf[RoundRobinPartitioning] && + newPartitioning.numPartitions > 1 + + // RDD passed to ShuffleDependency should be the form of key-value pairs. + // ColumnarShuffleWriter will compute ids from ColumnarBatch on native side other than read the "key" part. + // Thus in Columnar Shuffle we never use the "key" part. + val isOrderSensitive = isRoundRobin && !SQLConf.get.sortBeforeRepartition + + val rddWithDummyKey: RDD[Product2[Int, ColumnarBatch]] = newPartitioning match { + case RangePartitioning(sortingExpressions, _) => + rdd.mapPartitionsWithIndexInternal((_, cbIter) => { + val partitionKeyExtractor: InternalRow => Any = { + val projection = + UnsafeProjection.create(sortingExpressions.map(_.child), outputAttributes) + row => projection(row) + } + val newIter = computeAndAddPartitionId(cbIter, partitionKeyExtractor) + + SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit] { _ => + newIter.closeAppendedVector() + } + + newIter + }, isOrderSensitive = isOrderSensitive) + case _ => + rdd.mapPartitionsWithIndexInternal( + (_, cbIter) => + cbIter.map { cb => + (0 until cb.numCols).foreach( + cb.column(_) + .asInstanceOf[ArrowWritableColumnVector] + .getValueVector + .setValueCount(cb.numRows)) + (0, cb) + }, + isOrderSensitive = isOrderSensitive) + } + + val dependency = + new ColumnarShuffleDependency[Int, ColumnarBatch, ColumnarBatch]( + rddWithDummyKey, + new PartitionIdPassthrough(newPartitioning.numPartitions), + serializer, + shuffleWriterProcessor = + ShuffleExchangeExec.createShuffleWriteProcessor(writeMetrics), + nativePartitioning = nativePartitioning, + dataSize = dataSize, + bytesSpilled = bytesSpilled, + numInputRows = numInputRows, + computePidTime = computePidTime, + splitTime = splitTime, + spillTime = spillTime, + compressTime = compressTime, + prepareTime = prepareTime) + + dependency + } +} + + +case class CloseablePairedColumnarBatchIterator(iter: Iterator[(Int, ColumnarBatch)]) + extends Iterator[(Int, ColumnarBatch)] + with Logging { + + private var cur: (Int, ColumnarBatch) = _ + + def closeAppendedVector(): Unit = { + if (cur != null) { + logDebug("Close appended partition id vector") + cur match { + case (_, cb: ColumnarBatch) => + cb.column(0).asInstanceOf[ArrowWritableColumnVector].close() + } + cur = null + } + } + + override def hasNext: Boolean = { + iter.hasNext + } + + override def next(): (Int, ColumnarBatch) = { + closeAppendedVector() + if (iter.hasNext) { + cur = iter.next() + cur + } else Iterator.empty.next() + } +} \ No newline at end of file diff --git a/jvm/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala b/backends-velox/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala similarity index 100% rename from jvm/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala rename to backends-velox/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala diff --git a/dev/checkstyle-suppressions.xml b/dev/checkstyle-suppressions.xml new file mode 100644 index 000000000000..804a178a5fe2 --- /dev/null +++ b/dev/checkstyle-suppressions.xml @@ -0,0 +1,51 @@ + + + + + + + + + + + + + + + + + + diff --git a/dev/checkstyle.xml b/dev/checkstyle.xml new file mode 100644 index 000000000000..39b7b158cefb --- /dev/null +++ b/dev/checkstyle.xml @@ -0,0 +1,181 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/docs/ArrowBackend.md b/docs/ArrowBackend.md index 1068b9f0edc1..9b8b1d16bf6b 100644 --- a/docs/ArrowBackend.md +++ b/docs/ArrowBackend.md @@ -99,7 +99,7 @@ $ popd ```bash $ git clone https://github.com/oap-project/gluten.git $ pushd gluten -$ mvn clean package -P full-scala-compiler -DskipTests -Dcheckstyle.skip -Dbuild_cpp=ON -Dbuild_protobuf=ON -Dbuild_gazelle_cpp=ON -Darrow_root=${ARROW_HOME} +$ mvn clean package -Pbackends-gazelle -P full-scala-compiler -DskipTests -Dcheckstyle.skip -Dbuild_cpp=ON -Dbuild_protobuf=ON -Dbuild_gazelle_cpp=ON -Darrow_root=${ARROW_HOME} $ popd ``` diff --git a/docs/ClickHouse.md b/docs/ClickHouse.md index c91701856701..66c9e79d0141 100644 --- a/docs/ClickHouse.md +++ b/docs/ClickHouse.md @@ -114,7 +114,7 @@ The prerequisites are the same as the one above mentioned. Compile Gluten with C git clone https://github.com/oap-project/gluten.git cd gluten/ export MAVEN_OPTS="-Xmx8g -XX:ReservedCodeCacheSize=2g" - mvn clean install -Phadoop-2.7.4 -Pspark-3.1.1 -Dhadoop.version=2.8.5 -DskipTests -Dbuild_cpp=OFF -Dcpp_tests=OFF -Dbuild_arrow=ON -Dbuild_protobuf=ON -Dbuild_jemalloc=ON -Dcheckstyle.skip + mvn clean install -Pbackends-clickhouse -Phadoop-2.7.4 -Pspark-3.1.1 -Dhadoop.version=2.8.5 -DskipTests -Dbuild_cpp=OFF -Dcpp_tests=OFF -Dbuild_arrow=ON -Dbuild_protobuf=ON -Dbuild_jemalloc=ON -Dcheckstyle.skip ls -al jvm/target/gluten-jvm-XXXXX-jar-with-dependencies.jar ``` diff --git a/docs/GlutenUsage.md b/docs/GlutenUsage.md index 2db1aef10e6e..65d6083035a4 100644 --- a/docs/GlutenUsage.md +++ b/docs/GlutenUsage.md @@ -25,7 +25,7 @@ If you wish to enable Velox backend and you have an existing compiled Velox, ple The full compiling command would be like: ```shell script -mvn clean package -P full-scala-compiler -DskipTests -Dcheckstyle.skip -Dbuild_cpp=ON -Dbuild_velox=ON -Dbuild_velox_from_source=ON +mvn clean package -Pbackends-velox -P full-scala-compiler -DskipTests -Dcheckstyle.skip -Dbuild_cpp=ON -Dbuild_velox=ON -Dbuild_velox_from_source=ON ``` If Arrow has once been installed successfully on your env, and there is no change to Arrow, you can diff --git a/docs/Velox.md b/docs/Velox.md index a3b16a40ae74..643e39e4f081 100644 --- a/docs/Velox.md +++ b/docs/Velox.md @@ -48,7 +48,7 @@ After Velox being successfully compiled, please refer to [GlutenUsage](GlutenUsa use below command to compile Gluten with Velox backend. ```shell script -mvn clean package -P full-scala-compiler -DskipTests -Dcheckstyle.skip -Dbuild_cpp=ON -Dbuild_velox=ON -Dvelox_home=${VELOX_HOME} +mvn clean package -Pbackends-velox -P full-scala-compiler -DskipTests -Dcheckstyle.skip -Dbuild_cpp=ON -Dbuild_velox=ON -Dvelox_home=${VELOX_HOME} ``` ### An example for offloading Spark's computing to Velox with Gluten diff --git a/jvm/pom.xml b/jvm/pom.xml index 0b3790807d0b..f9ec97828a4e 100644 --- a/jvm/pom.xml +++ b/jvm/pom.xml @@ -24,7 +24,7 @@ gluten-jvm 1.0.0-snapshot jar - Gluten + Gluten-JVM ../cpp/ @@ -77,97 +77,6 @@ spark-hive-thriftserver_${scala.binary.version} provided - - io.delta - delta-core_${scala.binary.version} - provided - - - org.apache.arrow - ${arrow-memory.artifact} - ${arrow.version} - runtime - - - org.apache.arrow - arrow-memory-core - ${arrow.version} - - - io.netty - netty-common - - - io.netty - netty-buffer - - - - - - org.apache.arrow - arrow-vector - ${arrow.version} - - - io.netty - netty-common - - - io.netty - netty-buffer - - - - - - org.apache.arrow.gandiva - arrow-gandiva - ${arrow.version} - - - io.netty - netty-common - - - io.netty - netty-buffer - - - protobuf-java - com.google.protobuf - - - - - - 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 - - - protobuf-java - com.google.protobuf - - - compile - org.scalacheck scalacheck_${scala.binary.version} @@ -331,9 +240,6 @@ - - ${cpp.build.dir} - ${resource.dir} @@ -349,69 +255,6 @@ - - exec-maven-plugin - org.codehaus.mojo - 1.6.0 - - - Build arrow - generate-resources - - exec - - - bash - - ${arrow.script.dir}/build_arrow.sh - --tests=${cpp_tests} - --build_arrow=${build_arrow} - --static_arrow=${static_arrow} - --arrow_root=${arrow_root} - - - - - Build cpp - generate-resources - - exec - - - bash - - ${cpp.dir}/compile.sh - ${build_cpp} - ${cpp_tests} - ${jvm.build_arrow} - ${static_arrow} - ${build_protobuf} - ${arrow_root} - ${jvm.arrow.bfs.install.dir} - ${build_jemalloc} - ${build_gazelle_cpp} - ${build_velox} - ${velox_home} - - - - - Build velox - generate-resources - - exec - - - bash - - ${velox.script.dir}/build_velox.sh - --build_velox_from_source=${build_velox_from_source} - - - - - - org.apache.maven.plugins @@ -549,24 +392,6 @@ - - maven-assembly-plugin - 3.3.0 - - - jar-with-dependencies - - - - - make-assembly - package - - single - - - - org.scalastyle scalastyle-maven-plugin diff --git a/jvm/src/main/java/io/glutenproject/execution/AbstractColumnarNativeIterator.java b/jvm/src/main/java/io/glutenproject/execution/AbstractColumnarNativeIterator.java new file mode 100644 index 000000000000..2a3d7e028036 --- /dev/null +++ b/jvm/src/main/java/io/glutenproject/execution/AbstractColumnarNativeIterator.java @@ -0,0 +1,48 @@ +/* + * 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 io.glutenproject.execution; + +import org.apache.spark.sql.vectorized.ColumnarBatch; + +import java.util.Iterator; + +abstract public class AbstractColumnarNativeIterator implements Iterator, AutoCloseable { + protected final Iterator delegated; + protected ColumnarBatch nextBatch = null; + + public AbstractColumnarNativeIterator(Iterator delegated) { + this.delegated = delegated; + } + + @Override + public boolean hasNext() { + while (delegated.hasNext()) { + nextBatch = delegated.next(); + if (nextBatch.numRows() > 0) { + return true; + } + } + return false; + } + + public abstract byte[] next(); + + @Override + public void close() throws Exception { + } +} diff --git a/jvm/src/main/java/io/glutenproject/execution/CHColumnarNativeIterator.java b/jvm/src/main/java/io/glutenproject/execution/CHColumnarNativeIterator.java deleted file mode 100644 index 65af1c81dafb..000000000000 --- a/jvm/src/main/java/io/glutenproject/execution/CHColumnarNativeIterator.java +++ /dev/null @@ -1,40 +0,0 @@ -package io.glutenproject.execution; - -import io.glutenproject.expression.ConverterUtils; -import io.glutenproject.vectorized.CHColumnVector; -import io.glutenproject.vectorized.CHNativeBlock; -import org.apache.arrow.vector.ipc.message.ArrowRecordBatch; -import org.apache.spark.sql.vectorized.ColumnarBatch; - -import java.util.Iterator; - -public class CHColumnarNativeIterator extends ColumnarNativeIterator { - - public CHColumnarNativeIterator(Iterator delegated) { - super(delegated); - } - - @Override - public byte[] next() { - ColumnarBatch dep_cb = nextBatch; - if (dep_cb.numRows() > 0) { - CHColumnVector col = (CHColumnVector)dep_cb.column(0); - return longtoBytes(col.getBlockAddress()); - } else { - throw new IllegalStateException(); - } - } - - private static byte[] longtoBytes(long data) { - return new byte[]{ - (byte) ((data >> 56) & 0xff), - (byte) ((data >> 48) & 0xff), - (byte) ((data >> 40) & 0xff), - (byte) ((data >> 32) & 0xff), - (byte) ((data >> 24) & 0xff), - (byte) ((data >> 16) & 0xff), - (byte) ((data >> 8) & 0xff), - (byte) ((data >> 0) & 0xff), - }; - } -} diff --git a/jvm/src/main/java/io/glutenproject/vectorized/AbstractBatchIterator.java b/jvm/src/main/java/io/glutenproject/vectorized/AbstractBatchIterator.java new file mode 100644 index 000000000000..4df588e2aaa2 --- /dev/null +++ b/jvm/src/main/java/io/glutenproject/vectorized/AbstractBatchIterator.java @@ -0,0 +1,66 @@ +/* + * 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 io.glutenproject.vectorized; + +import java.io.IOException; + +import java.io.Serializable; + +abstract public class AbstractBatchIterator implements AutoCloseable, Serializable { + + protected long nativeHandler = 0; + protected boolean closed = false; + + public AbstractBatchIterator() throws IOException {} + + public AbstractBatchIterator(long instance_id) throws IOException { + nativeHandler = instance_id; + } + + public abstract boolean hasNextInternal() throws IOException; + + public boolean hasNext() throws IOException { + return hasNextInternal(); + } + + public abstract T nextInternal() throws IOException; + + public T next() throws IOException { + return nextInternal(); + } + + public abstract MetricsObject getMetricsInternal() throws IOException, ClassNotFoundException; + + public MetricsObject getMetrics() throws IOException, ClassNotFoundException { + if (nativeHandler == 0) { + return null; + } + return getMetricsInternal(); + } + + public abstract void closeInternal(); + + @Override + public void close() { + closeInternal(); + } + + long getInstanceId() { + return nativeHandler; + } +} diff --git a/jvm/src/main/java/io/glutenproject/vectorized/CHNativeBlock.java b/jvm/src/main/java/io/glutenproject/vectorized/CHNativeBlock.java deleted file mode 100644 index 7a399640bf36..000000000000 --- a/jvm/src/main/java/io/glutenproject/vectorized/CHNativeBlock.java +++ /dev/null @@ -1,86 +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 io.glutenproject.vectorized; - -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.vectorized.ColumnVector; -import org.apache.spark.sql.vectorized.ColumnarBatch; - -import java.util.Optional; -import java.util.stream.IntStream; - -public class CHNativeBlock { - private long blockAddress; - - public CHNativeBlock(long blockAddress) { - this.blockAddress = blockAddress; - } - - public static Optional fromColumnarBatch(ColumnarBatch batch) { - if (batch.numCols() == 0 || !(batch.column(0) instanceof CHColumnVector)) { - return Optional.empty(); - } - CHColumnVector columnVector = (CHColumnVector) batch.column(0); - return Optional.of(new CHNativeBlock(columnVector.getBlockAddress())); - } - - private native int nativeNumRows(long blockAddress); - - public int numRows() { - return nativeNumRows(blockAddress); - }; - - private native int nativeNumColumns(long blockAddress); - - public int numColumns() { - return nativeNumColumns(blockAddress); - } - - private native String nativeColumnType(long blockAddress, int position); - - public String getTypeByPosition(int position) { - return nativeColumnType(blockAddress, position); - } - - private native long nativeTotalBytes(long blockAddress); - - public long totalBytes() { - return nativeTotalBytes(blockAddress); - } - - public native void nativeClose(long blockAddress); - - public void close() { - if (blockAddress != 0) { - nativeClose(blockAddress); - blockAddress = 0; - } - } - - public ColumnarBatch toColumnarBatch() { - ColumnVector[] vectors = new ColumnVector[numColumns()]; - for (int i = 0; i < numColumns(); i++) { - vectors[i] = new CHColumnVector(DataType.fromDDL(getTypeByPosition(i)), blockAddress, i); - } - int numRows = 0; - if (numColumns() != 0) { - numRows = numRows(); - } - return new ColumnarBatch(vectors, numRows); - } -} diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ColumnarFactory.java b/jvm/src/main/java/io/glutenproject/vectorized/ColumnarFactory.java deleted file mode 100644 index 1d5bedda2c95..000000000000 --- a/jvm/src/main/java/io/glutenproject/vectorized/ColumnarFactory.java +++ /dev/null @@ -1,44 +0,0 @@ -package io.glutenproject.vectorized; - -import io.glutenproject.GlutenConfig; -import org.apache.spark.serializer.Serializer; -import org.apache.spark.sql.execution.metric.SQLMetric; -import org.apache.spark.sql.types.StructType; -import org.apache.spark.sql.vectorized.ColumnarBatch; -import scala.collection.Iterator; - -import java.io.IOException; - - -public class ColumnarFactory { - private ColumnarFactory() { - } - - public static boolean isEnableCH() { - return GlutenConfig.getConf().isClickHouseBackend(); - } - - public static Iterator createClosableIterator(Iterator iter) { - if (isEnableCH()) { - return new CloseableCHColumnBatchIterator(iter); - } else { - return new CloseableColumnBatchIterator(iter); - } - } - - public static Object createShuffleSplitterJniWrapper() throws IOException { - if (isEnableCH()) { - return new CHShuffleSplitterJniWrapper(); - } else { - return new ShuffleSplitterJniWrapper(); - } - } - - public static Serializer createColumnarBatchSerializer(StructType schema, SQLMetric readBatchNumRows, SQLMetric numOutputRows) { - if (isEnableCH()) { - return new CHColumnarBatchSerializer(readBatchNumRows, numOutputRows); - } else { - return new ArrowColumnarBatchSerializer(schema, readBatchNumRows, numOutputRows); - } - } -} diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ExpressionEvaluator.java b/jvm/src/main/java/io/glutenproject/vectorized/ExpressionEvaluator.java index 8f34f6c61339..a090d48ea7d9 100644 --- a/jvm/src/main/java/io/glutenproject/vectorized/ExpressionEvaluator.java +++ b/jvm/src/main/java/io/glutenproject/vectorized/ExpressionEvaluator.java @@ -18,23 +18,14 @@ package io.glutenproject.vectorized; import io.glutenproject.GlutenConfig; -import io.glutenproject.execution.ColumnarNativeIterator; +import io.glutenproject.backendsapi.BackendsApiManager; +import io.glutenproject.execution.AbstractColumnarNativeIterator; import io.glutenproject.row.RowIterator; import io.glutenproject.substrait.plan.PlanNode; -import org.apache.arrow.dataset.jni.NativeMemoryPool; -import org.apache.arrow.gandiva.exceptions.GandivaException; -import org.apache.arrow.gandiva.expression.ExpressionTree; -import org.apache.arrow.gandiva.ipc.GandivaTypes; -import org.apache.arrow.vector.ipc.WriteChannel; -import org.apache.arrow.vector.ipc.message.MessageSerializer; -import org.apache.arrow.vector.types.pojo.Schema; -import org.apache.commons.lang3.StringUtils; -import org.apache.spark.memory.MemoryConsumer; -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils; - -import java.io.ByteArrayOutputStream; + +import scala.collection.JavaConverters; + import java.io.IOException; -import java.nio.channels.Channels; import java.util.ArrayList; import java.util.List; @@ -95,25 +86,27 @@ public boolean doValidate(byte[] subPlan) { // Used by WholeStageTransfrom to create the native computing pipeline and // return a columnar result iterator. - public BatchIterator createKernelWithBatchIterator( - byte[] wsPlan, ArrayList iterList) + public AbstractBatchIterator createKernelWithBatchIterator( + byte[] wsPlan, ArrayList iterList) throws RuntimeException, IOException { - long poolId = 0; + /* long poolId = 0; if (!GlutenConfig.getConf().isClickHouseBackend()) { - NativeMemoryPool memoryPool = SparkMemoryUtils.contextMemoryPool(); - poolId = memoryPool.getNativeInstanceId(); + // NativeMemoryPool memoryPool = SparkMemoryUtils.contextMemoryPool(); + // poolId = memoryPool.getNativeInstanceId(); } ColumnarNativeIterator[] iterArray = new ColumnarNativeIterator[iterList.size()]; long batchIteratorInstance = jniWrapper.nativeCreateKernelWithIterator( - poolId, wsPlan, iterList.toArray(iterArray)); - return new BatchIterator(batchIteratorInstance); + poolId, wsPlan, iterList.toArray(iterArray)); */ + return BackendsApiManager.getIteratorApiInstance() + .genBatchIterator(wsPlan, + JavaConverters.asScalaIteratorConverter(iterList.iterator()).asScala().toSeq(), + jniWrapper); } - // Used by WholeStageTransfrom to create the native computing pipeline and // return a columnar result iterator. - public BatchIterator createKernelWithBatchIterator( - PlanNode wsPlan, ArrayList iterList) + public AbstractBatchIterator createKernelWithBatchIterator( + PlanNode wsPlan, ArrayList iterList) throws RuntimeException, IOException { return createKernelWithBatchIterator(getPlanBytesBuf(wsPlan), iterList); } @@ -122,7 +115,7 @@ public BatchIterator createKernelWithBatchIterator( // return a row result iterator. public RowIterator createKernelWithRowIterator( byte[] wsPlan, - ArrayList iterList) throws RuntimeException, IOException { + ArrayList iterList) throws RuntimeException, IOException { long rowIteratorInstance = jniWrapper.nativeCreateKernelWithRowIterator(wsPlan); return new RowIterator(rowIteratorInstance); } @@ -132,21 +125,6 @@ public void close() { jniWrapper.nativeClose(nativeHandler); } - byte[] getSchemaBytesBuf(Schema schema) throws IOException { - if (schema == null) return null; - ByteArrayOutputStream out = new ByteArrayOutputStream(); - MessageSerializer.serialize(new WriteChannel(Channels.newChannel(out)), schema); - return out.toByteArray(); - } - - byte[] getExprListBytesBuf(List exprs) throws GandivaException { - GandivaTypes.ExpressionList.Builder builder = GandivaTypes.ExpressionList.newBuilder(); - for (ExpressionTree expr : exprs) { - builder.addExprs(expr.toProtobuf()); - } - return builder.build().toByteArray(); - } - byte[] getPlanBytesBuf(PlanNode planNode) { return planNode.toProtobuf().toByteArray(); } diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ExpressionEvaluatorJniWrapper.java b/jvm/src/main/java/io/glutenproject/vectorized/ExpressionEvaluatorJniWrapper.java index 129c85e3acc2..13d8f90343b9 100644 --- a/jvm/src/main/java/io/glutenproject/vectorized/ExpressionEvaluatorJniWrapper.java +++ b/jvm/src/main/java/io/glutenproject/vectorized/ExpressionEvaluatorJniWrapper.java @@ -17,8 +17,7 @@ package io.glutenproject.vectorized; -import io.glutenproject.execution.ColumnarNativeIterator; -import org.apache.arrow.vector.ipc.message.ArrowRecordBatch; +import io.glutenproject.execution.AbstractColumnarNativeIterator; import org.apache.spark.memory.MemoryConsumer; import java.io.IOException; @@ -63,9 +62,10 @@ public ExpressionEvaluatorJniWrapper(String tmp_dir, List listJars, Stri * @param nativeHandler nativeHandler of this expression * @return iterator instance id */ - native long nativeCreateKernelWithIterator(long nativeHandler, + public native long nativeCreateKernelWithIterator(long nativeHandler, byte[] wsPlan, - ColumnarNativeIterator[] batchItr) throws RuntimeException; + AbstractColumnarNativeIterator[] batchItr + ) throws RuntimeException; /** * Create a native compute kernel and return a row iterator. diff --git a/jvm/src/main/scala/io/glutenproject/GlutenConfig.scala b/jvm/src/main/scala/io/glutenproject/GlutenConfig.scala index 49cc29cb9603..d3793f933eba 100644 --- a/jvm/src/main/scala/io/glutenproject/GlutenConfig.scala +++ b/jvm/src/main/scala/io/glutenproject/GlutenConfig.scala @@ -188,6 +188,9 @@ class GlutenConfig(conf: SQLConf) extends Logging { val isClickHouseBackend: Boolean = glutenBackendLib.equalsIgnoreCase(GlutenConfig.GLUTEN_CLICKHOUSE_BACKEND) + val isGazelleBackend: Boolean = + glutenBackendLib.equalsIgnoreCase(GlutenConfig.GLUTEN_GAZELLE_BACKEND) + // fallback to row operators if there are several continous joins val joinOptimizationThrottle: Integer = conf.getConfString("spark.gluten.sql.columnar.joinOptimizationLevel", "12").toInt @@ -255,7 +258,8 @@ object GlutenConfig { val GLUTEN_LOAD_ARROW = "spark.gluten.sql.columnar.loadarrow" val GLUTEN_BACKEND_LIB = "spark.gluten.sql.columnar.backend.lib" val GLUTEN_VELOX_BACKEND = "velox" - val GLUTEN_CLICKHOUSE_BACKEND = "clickhouse" + val GLUTEN_CLICKHOUSE_BACKEND = "ch" + val GLUTEN_GAZELLE_BACKEND = "gazelle_cpp" var ins: GlutenConfig = null var random_temp_dir_path: String = null diff --git a/jvm/src/main/scala/io/glutenproject/GlutenPlugin.scala b/jvm/src/main/scala/io/glutenproject/GlutenPlugin.scala index 3a267ce92d10..7f65f9a98c6c 100644 --- a/jvm/src/main/scala/io/glutenproject/GlutenPlugin.scala +++ b/jvm/src/main/scala/io/glutenproject/GlutenPlugin.scala @@ -22,6 +22,7 @@ import java.util.{Collections, Objects} import scala.language.implicitConversions import io.glutenproject.GlutenPlugin.{GLUTEN_SESSION_EXTENSION_NAME, SPARK_SESSION_EXTS_KEY} +import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.extension.{ColumnarOverrides, StrategyOverrides} import io.glutenproject.vectorized.ExpressionEvaluator import java.util @@ -45,6 +46,9 @@ private[glutenproject] class GlutenDriverPlugin extends DriverPlugin { override def init(sc: SparkContext, pluginContext: PluginContext): util.Map[String, String] = { val conf = pluginContext.conf() setPredefinedConfigs(conf) + // Initialize Backends API + BackendsApiManager.initialize(pluginContext.conf() + .get(GlutenConfig.GLUTEN_BACKEND_LIB, "")) Collections.emptyMap() } @@ -64,17 +68,18 @@ private[glutenproject] class GlutenExecutorPlugin extends ExecutorPlugin { override def init(ctx: PluginContext, extraConf: util.Map[String, String]): Unit = { // SQLConf is not initialed here, so it can not use 'GlutenConfig.getConf' to get conf. if (ctx.conf().getBoolean(GlutenConfig.GLUTEN_LOAD_NATIVE, defaultValue = true)) { - val customGLUTENLib = ctx.conf().get(GlutenConfig.GLUTEN_LIB_PATH, "") + val customGlutenLib = ctx.conf().get(GlutenConfig.GLUTEN_LIB_PATH, "") val customBackendLib = ctx.conf().get(GlutenConfig.GLUTEN_BACKEND_LIB, "") val initKernel = new ExpressionEvaluator(java.util.Collections.emptyList[String], ctx.conf().get(GlutenConfig.GLUTEN_LIB_NAME, "spark_columnar_jni"), - customGLUTENLib, + customGlutenLib, customBackendLib, ctx.conf().getBoolean(GlutenConfig.GLUTEN_LOAD_ARROW, defaultValue = true)) - if (customGLUTENLib.nonEmpty || customBackendLib.nonEmpty) { + if (customGlutenLib.nonEmpty || customBackendLib.nonEmpty) { initKernel.initNative() } } + BackendsApiManager.initialize(ctx.conf().get(GlutenConfig.GLUTEN_BACKEND_LIB, "")) } /** diff --git a/jvm/src/main/scala/io/glutenproject/backendsapi/BackendsApiManager.scala b/jvm/src/main/scala/io/glutenproject/backendsapi/BackendsApiManager.scala new file mode 100644 index 000000000000..a2c2751b257d --- /dev/null +++ b/jvm/src/main/scala/io/glutenproject/backendsapi/BackendsApiManager.scala @@ -0,0 +1,98 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.backendsapi + +import java.util.ServiceLoader +import scala.collection.JavaConversions._ + +object BackendsApiManager { + + protected var iteratorApiInstance: IIteratorApi = null + + protected var sparkPlanExecApiInstance: ISparkPlanExecApi = null + + protected var transformerApiInstance: ITransformerApi = null + + /** + * Initialize all backends api. + * + * @param glutenBackenLibName + */ + def initialize(glutenBackenLibName: String): Unit = synchronized { + // initialize IIteratorApi instance + if (iteratorApiInstance == null) { + val serviceLoader = ServiceLoader.load(classOf[IIteratorApi]) + for (ele <- serviceLoader) { + if (ele.getBackendName.equalsIgnoreCase(glutenBackenLibName)) { + iteratorApiInstance = ele + } else if (ele.getBackendName.equalsIgnoreCase("velox")) { + // Currently, Velox and Gazelle backends use the same api to implement. + // Will remove this after splitting gazelle module + iteratorApiInstance = ele + } + } + } + assert(iteratorApiInstance != null, "Can not initialize IIteratorApi instance.") + // initialize ISparkPlanExecApi instance + if (sparkPlanExecApiInstance == null) { + val serviceLoader = ServiceLoader.load(classOf[ISparkPlanExecApi]) + + for (ele <- serviceLoader) { + if (ele.getBackendName.equalsIgnoreCase(glutenBackenLibName)) { + sparkPlanExecApiInstance = ele + } else if (ele.getBackendName.equalsIgnoreCase("velox")) { + // Currently, Velox and Gazelle backends use the same api to implement. + // Will remove this after splitting gazelle module + sparkPlanExecApiInstance = ele + } + } + } + assert(sparkPlanExecApiInstance != null, "Can not initialize " + "ISparkPlanExecApi instance.") + // initialize ITransformerApi instance + if (transformerApiInstance == null) { + val serviceLoader = ServiceLoader.load(classOf[ITransformerApi]) + for (ele <- serviceLoader) { + if (ele.getBackendName.equalsIgnoreCase(glutenBackenLibName)) { + transformerApiInstance = ele + } else if (ele.getBackendName.equalsIgnoreCase("velox")) { + // Currently, Velox and Gazelle backends use the same api to implement. + // Will remove this after splitting gazelle module + transformerApiInstance = ele + } + } + } + assert(transformerApiInstance != null, "Can not initialize ITransformerApi instance.") + } + + def getIteratorApiInstance: IIteratorApi = { + if (iteratorApiInstance == null) + throw new RuntimeException("IIteratorApi instance is null.") + iteratorApiInstance + } + + def getSparkPlanExecApiInstance: ISparkPlanExecApi = { + if (sparkPlanExecApiInstance == null) + throw new RuntimeException("ISparkPlanExecApi instance is null.") + sparkPlanExecApiInstance + } + + def getTransformerApiInstance: ITransformerApi = { + if (transformerApiInstance == null) + throw new RuntimeException("ITransformerApi instance is null.") + transformerApiInstance + } +} diff --git a/jvm/src/main/scala/io/glutenproject/backendsapi/IBackendsApi.scala b/jvm/src/main/scala/io/glutenproject/backendsapi/IBackendsApi.scala new file mode 100644 index 000000000000..19328caf5bd1 --- /dev/null +++ b/jvm/src/main/scala/io/glutenproject/backendsapi/IBackendsApi.scala @@ -0,0 +1,27 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.backendsapi + +trait IBackendsApi { + + /** + * Get the backend api name. + * + * @return + */ + def getBackendName: String +} diff --git a/jvm/src/main/scala/io/glutenproject/backendsapi/IIteratorApi.scala b/jvm/src/main/scala/io/glutenproject/backendsapi/IIteratorApi.scala new file mode 100644 index 000000000000..5d6baa29ffa6 --- /dev/null +++ b/jvm/src/main/scala/io/glutenproject/backendsapi/IIteratorApi.scala @@ -0,0 +1,97 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.backendsapi + +import io.glutenproject.GlutenNumaBindingInfo +import io.glutenproject.execution.{AbstractColumnarNativeIterator, BaseNativeFilePartition, WholestageTransformContext} +import io.glutenproject.substrait.plan.PlanNode +import io.glutenproject.vectorized.{AbstractBatchIterator, ExpressionEvaluator, ExpressionEvaluatorJniWrapper} +import org.apache.spark.{SparkConf, TaskContext} + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.vectorized.ColumnarBatch + +trait IIteratorApi extends IBackendsApi { + + /** + * Generate native row partition. + * + * @return + */ + def genNativeFilePartition(p: InputPartition, + wsCxt: WholestageTransformContext): BaseNativeFilePartition + + /** + * Generate Iterator[ColumnarBatch] for CoalesceBatchesExec. + * + * @return + */ + def genCoalesceIterator(iter: Iterator[ColumnarBatch], + recordsPerBatch: Int, numOutputRows: SQLMetric, + numInputBatches: SQLMetric, numOutputBatches: SQLMetric, + collectTime: SQLMetric, concatTime: SQLMetric, + avgCoalescedNumRows: SQLMetric): Iterator[ColumnarBatch] + + /** + * Generate closeable ColumnBatch iterator. + * + * @return + */ + def genCloseableColumnBatchIterator(iter: Iterator[ColumnarBatch]): Iterator[ColumnarBatch] + + /** + * Generate Iterator[ColumnarBatch] for first stage. + * + * @return + */ + def genFirstStageIterator(inputPartition: BaseNativeFilePartition, loadNative: Boolean, + outputAttributes: Seq[Attribute], context: TaskContext, + jarList: Seq[String]): Iterator[ColumnarBatch] + + /** + * Generate Iterator[ColumnarBatch] for final stage. + * + * @return + */ + def genFinalStageIterator(iter: Iterator[ColumnarBatch], + numaBindingInfo: GlutenNumaBindingInfo, listJars: Seq[String], + signature: String, sparkConf: SparkConf, + outputAttributes: Seq[Attribute], rootNode: PlanNode, + streamedSortPlan: SparkPlan, pipelineTime: SQLMetric, + buildRelationBatchHolder: Seq[ColumnarBatch], + dependentKernels: Seq[ExpressionEvaluator], + dependentKernelIterators: Seq[AbstractBatchIterator] + ): Iterator[ColumnarBatch] + + /** + * Generate columnar native iterator. + * + * @return + */ + def genColumnarNativeIterator(delegated: Iterator[ColumnarBatch]): AbstractColumnarNativeIterator + + /** + * Generate BatchIterator for ExpressionEvaluator. + * + * @return + */ + def genBatchIterator(wsPlan: Array[Byte], iterList: Seq[AbstractColumnarNativeIterator], + jniWrapper: ExpressionEvaluatorJniWrapper): AbstractBatchIterator +} diff --git a/jvm/src/main/scala/io/glutenproject/backendsapi/ISparkPlanExecApi.scala b/jvm/src/main/scala/io/glutenproject/backendsapi/ISparkPlanExecApi.scala new file mode 100644 index 000000000000..6d268c8daa19 --- /dev/null +++ b/jvm/src/main/scala/io/glutenproject/backendsapi/ISparkPlanExecApi.scala @@ -0,0 +1,78 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.backendsapi + +import io.glutenproject.execution.{NativeColumnarToRowExec, RowToArrowColumnarExec} + +import org.apache.spark.ShuffleDependency +import org.apache.spark.rdd.RDD +import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.{GenShuffleWriterParameters, GlutenShuffleWriterWrapper} +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch + +trait ISparkPlanExecApi extends IBackendsApi { + + /** + * Generate NativeColumnarToRowExec. + * + * @param child + * @return + */ + def genNativeColumnarToRowExec(child: SparkPlan): NativeColumnarToRowExec + + /** + * Generate RowToArrowColumnarExec. + * + * @param child + * @return + */ + def genRowToArrowColumnarExec(child: SparkPlan): RowToArrowColumnarExec + + /** + * Generate ShuffleDependency for ColumnarShuffleExchangeExec. + * + * @return + */ + def genShuffleDependency(rdd: RDD[ColumnarBatch], outputAttributes: Seq[Attribute], + newPartitioning: Partitioning, serializer: Serializer, + writeMetrics: Map[String, SQLMetric], dataSize: SQLMetric, + bytesSpilled: SQLMetric, numInputRows: SQLMetric, + computePidTime: SQLMetric, splitTime: SQLMetric, + spillTime: SQLMetric, compressTime: SQLMetric, prepareTime: SQLMetric + ): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] + + /** + * Generate ColumnarShuffleWriter for ColumnarShuffleManager. + * + * @return + */ + def genColumnarShuffleWriter[K, V](parameters: GenShuffleWriterParameters[K, V] + ): GlutenShuffleWriterWrapper[K, V] + + /** + * Generate ColumnarBatchSerializer for ColumnarShuffleExchangeExec. + * + * @return + */ + def createColumnarBatchSerializer(schema: StructType, readBatchNumRows: SQLMetric, + numOutputRows: SQLMetric): Serializer +} diff --git a/jvm/src/main/scala/io/glutenproject/backendsapi/ITransformerApi.scala b/jvm/src/main/scala/io/glutenproject/backendsapi/ITransformerApi.scala new file mode 100644 index 000000000000..1d2a5fbe830d --- /dev/null +++ b/jvm/src/main/scala/io/glutenproject/backendsapi/ITransformerApi.scala @@ -0,0 +1,31 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.backendsapi + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.Partitioning + +trait ITransformerApi extends IBackendsApi { + + /** + * Do validate for ColumnarShuffleExchangeExec. + * + * @return + */ + def validateColumnarShuffleExchangeExec(outputPartitioning: Partitioning, + outputAttributes: Seq[Attribute]): Boolean +} diff --git a/jvm/src/main/scala/io/glutenproject/execution/BroadcastHashJoinExecTransformer.scala b/jvm/src/main/scala/io/glutenproject/execution/BroadcastHashJoinExecTransformer.scala index 6ea4d17d5877..5e409da56a39 100644 --- a/jvm/src/main/scala/io/glutenproject/execution/BroadcastHashJoinExecTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/execution/BroadcastHashJoinExecTransformer.scala @@ -17,33 +17,20 @@ package io.glutenproject.execution -import com.google.common.collect.Lists -import io.glutenproject.GlutenConfig -import io.glutenproject.expression._ import io.glutenproject.substrait.SubstraitContext -import io.glutenproject.substrait.rel.RelNode -import io.glutenproject.vectorized.{ExpressionEvaluator, _} -import org.apache.arrow.gandiva.expression._ -import org.apache.arrow.vector.types.pojo.{ArrowType, Field} + import org.apache.spark.rdd.RDD 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.joins.{BaseJoinExec, HashJoin, ShuffledJoin} -import org.apache.spark.sql.execution.joins.HashedRelationInfo -import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.util.ArrowUtils -import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} -import org.apache.spark.util.{ExecutorManager, UserAddedJarUtils} -import org.apache.spark.sql.types.DecimalType +import org.apache.spark.sql.vectorized.ColumnarBatch -import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.collection.mutable.ListBuffer /** * Performs a hash join of two child relations by first shuffling the data using the join keys. diff --git a/jvm/src/main/scala/io/glutenproject/execution/CoalesceBatchesExec.scala b/jvm/src/main/scala/io/glutenproject/execution/CoalesceBatchesExec.scala index 1967b3b50d8c..b444fcbe5228 100644 --- a/jvm/src/main/scala/io/glutenproject/execution/CoalesceBatchesExec.scala +++ b/jvm/src/main/scala/io/glutenproject/execution/CoalesceBatchesExec.scala @@ -17,24 +17,15 @@ package io.glutenproject.execution -import scala.collection.mutable.ListBuffer -import scala.collection.JavaConverters._ - -import io.glutenproject.GlutenConfig -import io.glutenproject.vectorized.{ArrowWritableColumnVector, CHCoalesceOperator, CHNativeBlock, ColumnarFactory} -import org.apache.arrow.vector.types.pojo.Schema -import org.apache.arrow.vector.util.VectorBatchAppender -import org.apache.spark.TaskContext +import io.glutenproject.backendsapi.BackendsApiManager 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.{SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.util.ArrowUtils -import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.sql.vectorized.ColumnarBatch case class CoalesceBatchesExec(child: SparkPlan) extends UnaryExecNode { @@ -58,16 +49,6 @@ case class CoalesceBatchesExec(child: SparkPlan) extends UnaryExecNode { .createAverageMetric(sparkContext, "avg coalesced batch num rows")) override def doExecuteColumnar(): RDD[ColumnarBatch] = { - if (GlutenConfig.getConf.isClickHouseBackend) { - doCHInternalExecuteColumnar() - } else { - doInternalExecuteColumnar() - } - } - - def doInternalExecuteColumnar(): RDD[ColumnarBatch] = { - import CoalesceBatchesExec._ - val recordsPerBatch = conf.arrowMaxRecordsPerBatch val numOutputRows = longMetric("numOutputRows") val numInputBatches = longMetric("numInputBatches") @@ -77,142 +58,9 @@ case class CoalesceBatchesExec(child: SparkPlan) extends UnaryExecNode { val avgCoalescedNumRows = longMetric("avgCoalescedNumRows") child.executeColumnar().mapPartitions { iter => - val beforeInput = System.nanoTime - val hasInput = iter.hasNext - collectTime += System.nanoTime - beforeInput - val res = if (hasInput) { - new Iterator[ColumnarBatch] { - var numBatchesTotal: Long = _ - var numRowsTotal: Long = _ - SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit] { _ => - if (numBatchesTotal > 0) { - avgCoalescedNumRows.set(numRowsTotal.toDouble / numBatchesTotal) - } - } - - override def hasNext: Boolean = { - val beforeNext = System.nanoTime - val hasNext = iter.hasNext - collectTime += System.nanoTime - beforeNext - hasNext - } - - override def next(): ColumnarBatch = { - - if (!hasNext) { - throw new NoSuchElementException("End of ColumnarBatch iterator") - } - - var rowCount = 0 - val batchesToAppend = ListBuffer[ColumnarBatch]() - - while (hasNext && rowCount < recordsPerBatch) { - val delta = iter.next() - delta.retain() - rowCount += delta.numRows - batchesToAppend += delta - } - - // chendi: We need make sure target FieldTypes are exactly the same as src - val expected_output_arrow_fields = if (batchesToAppend.size > 0) { - (0 until batchesToAppend(0).numCols).map(i => { - batchesToAppend(0).column(i).asInstanceOf[ArrowWritableColumnVector].getValueVector.getField - }) - } else { - Nil - } - - val resultStructType = ArrowUtils.fromArrowSchema(new Schema(expected_output_arrow_fields.asJava)) - val beforeConcat = System.nanoTime - val resultColumnVectors = - ArrowWritableColumnVector.allocateColumns(rowCount, resultStructType).toArray - val target = - new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), rowCount) - coalesce(target, batchesToAppend.toList) - target.setNumRows(rowCount) - - concatTime += System.nanoTime - beforeConcat - numOutputRows += rowCount - numInputBatches += batchesToAppend.length - numOutputBatches += 1 - - // used for calculating avgCoalescedNumRows - numRowsTotal += rowCount - numBatchesTotal += 1 - - batchesToAppend.foreach(cb => cb.close()) - - target - } - } - } else { - Iterator.empty - } - ColumnarFactory.createClosableIterator(res) - } - } - - def doCHInternalExecuteColumnar(): RDD[ColumnarBatch] = { - val recordsPerBatch = conf.arrowMaxRecordsPerBatch - val numOutputRows = longMetric("numOutputRows") - val numInputBatches = longMetric("numInputBatches") - val numOutputBatches = longMetric("numOutputBatches") - val collectTime = longMetric("collectTime") - val concatTime = longMetric("concatTime") - val avgCoalescedNumRows = longMetric("avgCoalescedNumRows") - child.executeColumnar().mapPartitions(iter => { - val operator = new CHCoalesceOperator(recordsPerBatch) - val res = new Iterator[ColumnarBatch] { - override def hasNext: Boolean = { - val beforeNext = System.nanoTime - val hasNext = iter.hasNext - collectTime += System.nanoTime - beforeNext - hasNext - } - SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit]((tc: TaskContext) => { - operator.close() - }) - override def next(): ColumnarBatch = { - val c = iter.next() - numInputBatches += 1 - val beforeConcat = System.nanoTime - operator.mergeBlock(c) - - while(!operator.isFull && iter.hasNext) { - val cb = iter.next(); - numInputBatches += 1; - operator.mergeBlock(cb) - } - val res = operator.release().toColumnarBatch - CHNativeBlock.fromColumnarBatch(res).ifPresent(block => { - numOutputRows += block.numRows(); - numOutputBatches += 1; - }) - res - } - } - - ColumnarFactory.createClosableIterator(res) - }) - } -} - -object CoalesceBatchesExec { - implicit class ArrowColumnarBatchRetainer(val cb: ColumnarBatch) { - def retain(): Unit = { - (0 until cb.numCols).toList.foreach(i => - cb.column(i).asInstanceOf[ArrowWritableColumnVector].retain()) - } - } - - def coalesce(targetBatch: ColumnarBatch, batchesToAppend: List[ColumnarBatch]): Unit = { - (0 until targetBatch.numCols).toList.foreach { i => - val targetVector = - targetBatch.column(i).asInstanceOf[ArrowWritableColumnVector].getValueVector - val vectorsToAppend = batchesToAppend.map { cb => - cb.column(i).asInstanceOf[ArrowWritableColumnVector].getValueVector - } - VectorBatchAppender.batchAppend(targetVector, vectorsToAppend: _*) + BackendsApiManager.getIteratorApiInstance + .genCoalesceIterator(iter, recordsPerBatch, numOutputRows, + numInputBatches, numOutputBatches, collectTime, concatTime, avgCoalescedNumRows) } } } diff --git a/jvm/src/main/scala/io/glutenproject/execution/ExpandExecTransformer.scala b/jvm/src/main/scala/io/glutenproject/execution/ExpandExecTransformer.scala index b01f70fddd26..369cc06c1075 100644 --- a/jvm/src/main/scala/io/glutenproject/execution/ExpandExecTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/execution/ExpandExecTransformer.scala @@ -17,22 +17,15 @@ package io.glutenproject.execution -import io.glutenproject.expression._ import io.glutenproject.substrait.SubstraitContext -import io.glutenproject.vectorized._ -import org.apache.arrow.vector.ValueVector -import org.apache.spark.internal.Logging + import org.apache.spark.rdd.RDD -import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} -import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} -import org.apache.spark.sql.types.{StructField, StructType} -import org.apache.spark.TaskContext -import org.apache.spark.sql.types.DecimalType +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.vectorized.ColumnarBatch case class ExpandExecTransformer( projections: Seq[Seq[Expression]], diff --git a/jvm/src/main/scala/io/glutenproject/execution/HashAggregateExecTransformer.scala b/jvm/src/main/scala/io/glutenproject/execution/HashAggregateExecTransformer.scala index 80641fc801cf..ef4f0ce1dc57 100644 --- a/jvm/src/main/scala/io/glutenproject/execution/HashAggregateExecTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/execution/HashAggregateExecTransformer.scala @@ -18,19 +18,21 @@ package io.glutenproject.execution import scala.collection.mutable.ListBuffer +import scala.util.control.Breaks.{break, breakable} + import com.google.common.collect.Lists +import com.google.protobuf.Any import io.glutenproject.expression._ import io.glutenproject.substrait.expression.{AggregateFunctionNode, ExpressionBuilder, ExpressionNode} import io.glutenproject.substrait.rel.{LocalFilesBuilder, RelBuilder, RelNode} import io.glutenproject.substrait.SubstraitContext import io.glutenproject.GlutenConfig -import java.util - -import com.google.protobuf.Any import io.glutenproject.substrait.`type`.{TypeBuilder, TypeNode} import io.glutenproject.substrait.extensions.ExtensionBuilder import io.glutenproject.substrait.plan.PlanBuilder import io.glutenproject.vectorized.ExpressionEvaluator +import java.util + import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -41,8 +43,6 @@ import org.apache.spark.sql.execution.aggregate._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.vectorized.ColumnarBatch -import scala.util.control.Breaks.{break, breakable} - /** * Columnar Based HashAggregateExec. */ @@ -94,6 +94,15 @@ case class HashAggregateExecTransformer( numOutputBatches.set(0) numInputBatches.set(0) + lazy val aggregateResultAttributes = { + val groupingAttributes = groupingExpressions.map(expr => { + ConverterUtils.getAttrFromExpr(expr).toAttribute + }) + groupingAttributes ++ aggregateExpressions.map(expr => { + expr.resultAttribute + }) + } + override def doValidate(): Boolean = { val substraitContext = new SubstraitContext val relNode = try { @@ -167,20 +176,41 @@ case class HashAggregateExecTransformer( case _ => null } - val (relNode, inputAttributes) = if (childCtx != null) { - (getAggRel(context.registeredFunction, childCtx.root), childCtx.outputAttributes) + val (relNode, inputAttributes, outputAttributes) = if (childCtx != null) { + if (!GlutenConfig.getConf.isClickHouseBackend) { + (getAggRel(context.registeredFunction, childCtx.root), + childCtx.outputAttributes, output) + } else { + (getAggRel(context.registeredFunction, childCtx.root), + childCtx.outputAttributes, aggregateResultAttributes) + } } else { // This means the input is just an iterator, so an ReadRel will be created as child. // Prepare the input schema. - val attrList = new util.ArrayList[Attribute]() - for (attr <- child.output) { - attrList.add(attr) - } - val readRel = RelBuilder.makeReadRel(attrList, context) + if (!GlutenConfig.getConf.isClickHouseBackend) { + val attrList = new util.ArrayList[Attribute]() + for (attr <- child.output) { + attrList.add(attr) + } + val readRel = RelBuilder.makeReadRel(attrList, context) + (getAggRel(context.registeredFunction, readRel), child.output, output) + } else { + val typeList = new util.ArrayList[TypeNode]() + val nameList = new util.ArrayList[String]() + for (attr <- aggregateResultAttributes) { + typeList.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) + nameList.add(ConverterUtils.getShortAttributeName(attr) + "#" + attr.exprId.id) + } + // The iterator index will be added in the path of LocalFiles. + val inputIter = LocalFilesBuilder.makeLocalFiles( + ConverterUtils.ITERATOR_PREFIX.concat(context.getIteratorIndex.toString)) + context.setLocalFilesNode(inputIter) + val readRel = RelBuilder.makeReadRel(typeList, nameList, context) - (getAggRel(context.registeredFunction, readRel), child.output) + (getAggRel(context.registeredFunction, readRel), aggregateResultAttributes, output) + } } - TransformContext(inputAttributes, output, relNode) + TransformContext(inputAttributes, outputAttributes, relNode) } override def verboseString(maxFields: Int): String = toString(verbose = true, maxFields) @@ -362,11 +392,17 @@ case class HashAggregateExecTransformer( aggExpr.asInstanceOf[ExpressionTransformer].doTransform(args) }) case Final => - aggregatFunc.inputAggBufferAttributes.toList.map(attr => { - val aggExpr: Expression = ExpressionConverter - .replaceWithExpressionTransformer(attr, originalInputAttributes) - aggExpr.asInstanceOf[ExpressionTransformer].doTransform(args) - }) + if (!GlutenConfig.getConf.isClickHouseBackend) { + aggregatFunc.inputAggBufferAttributes.toList.map(attr => { + val aggExpr: Expression = ExpressionConverter + .replaceWithExpressionTransformer(attr, originalInputAttributes) + aggExpr.asInstanceOf[ExpressionTransformer].doTransform(args) + }) + } else { + val aggTypesExpr: Expression = ExpressionConverter + .replaceWithExpressionTransformer(aggExpr.resultAttribute, originalInputAttributes) + Seq(aggTypesExpr.asInstanceOf[ExpressionTransformer].doTransform(args)) + } case other => throw new UnsupportedOperationException(s"$other not supported.") } @@ -400,7 +436,11 @@ case class HashAggregateExecTransformer( val aggRel = if (needsPreProjection) { getAggRelWithPreProjection(args, originalInputAttributes, input, validation) } else { - getAggRelWithoutPreProjection(args, originalInputAttributes, input, validation) + if (!GlutenConfig.getConf.isClickHouseBackend) { + getAggRelWithoutPreProjection(args, originalInputAttributes, input, validation) + } else { + getAggRelWithoutPreProjection(args, aggregateResultAttributes, input, validation) + } } // Will check if post-projection is needed. If yes, a ProjectRel will be added after the // AggregateRel. diff --git a/jvm/src/main/scala/io/glutenproject/execution/NativeColumnarToRowExec.scala b/jvm/src/main/scala/io/glutenproject/execution/NativeColumnarToRowExec.scala index dd7f1dc68234..fcc4bffbcde3 100644 --- a/jvm/src/main/scala/io/glutenproject/execution/NativeColumnarToRowExec.scala +++ b/jvm/src/main/scala/io/glutenproject/execution/NativeColumnarToRowExec.scala @@ -17,27 +17,12 @@ package io.glutenproject.execution -import io.glutenproject.GlutenConfig -import io.glutenproject.expression.ConverterUtils -import io.glutenproject.vectorized.{ArrowWritableColumnVector, NativeColumnarToRowJniWrapper} -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} -import org.apache.spark.sql.execution.datasources.v2.BatchScanExec -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.{ColumnarToRowExec, SparkPlan} -import org.apache.spark.sql.types._ - -import scala.collection.JavaConverters._ -import scala.collection.mutable.ListBuffer -import scala.concurrent.duration._ - -class NativeColumnarToRowExec(child: SparkPlan) extends ColumnarToRowExec(child = child) { - override def nodeName: String = "NativeColumnarToRow" +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} - override def supportCodegen: Boolean = false +abstract class NativeColumnarToRowExec(child: SparkPlan) extends ColumnarToRowExec(child = child) { // A flag used to check whether child is wholestage transformer. // Different backends may have different behaviours according to this flag. @@ -54,27 +39,7 @@ class NativeColumnarToRowExec(child: SparkPlan) extends ColumnarToRowExec(child true } - def buildCheck(): Unit = { - val schema = child.schema - for (field <- schema.fields) { - field.dataType match { - case d: BooleanType => - case d: ByteType => - case d: ShortType => - case d: IntegerType => - case d: LongType => - case d: FloatType => - case d: DoubleType => - case d: StringType => - case d: DateType => - case d: DecimalType => - case d: TimestampType => - case d: BinaryType => - case _ => - throw new UnsupportedOperationException(s"${field.dataType} is not supported in NativeColumnarToRowExec.") - } - } - } + def buildCheck(): Unit override lazy val metrics: Map[String, SQLMetric] = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), @@ -82,117 +47,10 @@ class NativeColumnarToRowExec(child: SparkPlan) extends ColumnarToRowExec(child "convertTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to convert") ) - override def doExecute(): RDD[InternalRow] = { - val numOutputRows = longMetric("numOutputRows") - val numInputBatches = longMetric("numInputBatches") - val convertTime = longMetric("convertTime") - - // Fake Arrow format will be returned for WS transformer. - // TODO: use a Velox layer on the top of the base layer. - if (GlutenConfig.getConf.isVeloxBackend) { - if (child.isInstanceOf[WholeStageTransformerExec]) { - child.asInstanceOf[WholeStageTransformerExec].setFakeOutput() - } - } - - child.executeColumnar().mapPartitions { batches => - // TODO:: pass the jni jniWrapper and arrowSchema and serializeSchema method by broadcast - val jniWrapper = new NativeColumnarToRowJniWrapper() - var arrowSchema: Array[Byte] = null - - def serializeSchema(fields: Seq[Field]): Array[Byte] = { - val schema = new Schema(fields.asJava) - ConverterUtils.getSchemaBytesBuf(schema) - } - - batches.flatMap { batch => - numInputBatches += 1 - numOutputRows += batch.numRows() + def doExecuteInternal(): RDD[InternalRow] - if (batch.numRows == 0) { - logInfo(s"Skip ColumnarBatch of ${batch.numRows} rows, ${batch.numCols} cols") - Iterator.empty - } else if (this.output.isEmpty || (batch.numCols() > 0 && - !batch.column(0).isInstanceOf[ArrowWritableColumnVector])) { - // Fallback to ColumnarToRow - val localOutput = this.output - numInputBatches += 1 - numOutputRows += batch.numRows() - - val toUnsafe = UnsafeProjection.create(localOutput, localOutput) - batch.rowIterator().asScala.map(toUnsafe) - } else { - val bufAddrs = new ListBuffer[Long]() - val bufSizes = new ListBuffer[Long]() - val fields = new ListBuffer[Field]() - (0 until batch.numCols).foreach { idx => - val column = batch.column(idx).asInstanceOf[ArrowWritableColumnVector] - fields += column.getValueVector.getField - column.getValueVector.getBuffers(false) - .foreach { buffer => - bufAddrs += buffer.memoryAddress() - try { - bufSizes += buffer.readableBytes() - } catch { - case e: Throwable => - // For Velox, the returned format is faked arrow format, - // and the offset buffer is invalid. Only the buffer address is cared. - if (GlutenConfig.getConf.isVeloxBackend && - child.output(idx).dataType == StringType) { - // Add a fake value here for String column. - bufSizes += 1 - } else { - throw e - } - } - } - } - - if (arrowSchema == null) { - arrowSchema = serializeSchema(fields) - } - - val beforeConvert = System.nanoTime() - - val info = jniWrapper.nativeConvertColumnarToRow( - arrowSchema, batch.numRows, bufAddrs.toArray, bufSizes.toArray, - SparkMemoryUtils.contextMemoryPool().getNativeInstanceId, wsChild) - - convertTime += NANOSECONDS.toMillis(System.nanoTime() - beforeConvert) - - new Iterator[InternalRow] { - var rowId = 0 - val row = new UnsafeRow(batch.numCols()) - var closed = false - override def hasNext: Boolean = { - val result = rowId < batch.numRows() - if (!result && !closed) { - jniWrapper.nativeClose(info.instanceID) - closed = true - } - return result - } - - override def next: UnsafeRow = { - if (rowId >= batch.numRows()) throw new NoSuchElementException - - val (offset, length) = (info.offsets(rowId), info.lengths(rowId)) - row.pointTo(null, info.memoryAddress + offset, length.toInt) - rowId += 1 - row - } - } - } - } - } - } - - override def canEqual(other: Any): Boolean = other.isInstanceOf[NativeColumnarToRowExec] - - override def equals(other: Any): Boolean = other match { - case that: NativeColumnarToRowExec => - (that canEqual this) && super.equals(that) - case _ => false + override def doExecute(): RDD[InternalRow] = { + doExecuteInternal() } } diff --git a/jvm/src/main/scala/io/glutenproject/execution/NativeWholeStageColumnarRDD.scala b/jvm/src/main/scala/io/glutenproject/execution/NativeWholeStageColumnarRDD.scala index ef7b676f1580..6926e8ed2ea3 100644 --- a/jvm/src/main/scala/io/glutenproject/execution/NativeWholeStageColumnarRDD.scala +++ b/jvm/src/main/scala/io/glutenproject/execution/NativeWholeStageColumnarRDD.scala @@ -17,49 +17,28 @@ package io.glutenproject.execution -import io.glutenproject.vectorized.ColumnarFactory - import java.io.Serializable import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ListBuffer +import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.GlutenConfig -import io.glutenproject.expression.ConverterUtils import io.glutenproject.vectorized._ -import org.apache.arrow.vector.types.pojo.Schema import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.execution.datasources.PartitionedFile -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils -import org.apache.spark.sql.util.ArrowUtils -import org.apache.spark.sql.util.OASPackageBridge._ -import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util._ trait BaseNativeFilePartition extends Partition with InputPartition { def substraitPlan: Array[Byte] } -case class NativeMergeTreePartition(index: Int, engine: String, - database: String, - table: String, tablePath: String, - minParts: Long, maxParts: Long, - substraitPlan: Array[Byte] = Array.empty[Byte]) - extends BaseNativeFilePartition { - override def preferredLocations(): Array[String] = { - Array.empty[String] - } - - def copySubstraitPlan(newSubstraitPlan: Array[Byte]): NativeMergeTreePartition = { - this.copy(substraitPlan = newSubstraitPlan) - } -} - case class NativeFilePartition(index: Int, files: Array[PartitionedFile], substraitPlan: Array[Byte]) extends BaseNativeFilePartition { @@ -90,7 +69,7 @@ class NativeWholeStageColumnarRDD( columnarReads: Boolean, outputAttributes: Seq[Attribute], jarList: Seq[String], - dependentKernelIterators: ListBuffer[BatchIterator]) + dependentKernelIterators: ListBuffer[AbstractBatchIterator]) extends RDD[ColumnarBatch](sc, Nil) { val numaBindingInfo = GlutenConfig.getConf.numaBindingInfo val loadNative: Boolean = GlutenConfig.getConf.loadNative @@ -107,10 +86,7 @@ class NativeWholeStageColumnarRDD( } private def castNativePartition(split: Partition): BaseNativeFilePartition = split match { - case NativeSubstraitPartition(_, p: NativeFilePartition) => - p.asInstanceOf[BaseNativeFilePartition] - case NativeSubstraitPartition(_, p: NativeMergeTreePartition) => - p.asInstanceOf[BaseNativeFilePartition] + case NativeSubstraitPartition(_, p: BaseNativeFilePartition) => p case _ => throw new SparkException(s"[BUG] Not a NativeSubstraitPartition: $split") } @@ -119,73 +95,8 @@ class NativeWholeStageColumnarRDD( val inputPartition = castNativePartition(split) - var inputSchema : Schema = null - var outputSchema : Schema = null - var resIter : BatchIterator = null - if (loadNative) { - // TODO: 'jarList' is kept for codegen - val transKernel = new ExpressionEvaluator(jarList.toList.asJava) - val inBatchIters = new java.util.ArrayList[ColumnarNativeIterator]() - outputSchema = ConverterUtils.toArrowSchema(outputAttributes) - resIter = transKernel.createKernelWithBatchIterator( - inputPartition.substraitPlan, inBatchIters) - SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit] { _ => resIter.close() } - } - val iter = new Iterator[Any] { - private val inputMetrics = TaskContext.get().taskMetrics().inputMetrics - - override def hasNext: Boolean = { - if (loadNative) { - resIter.hasNext - } else { - false - } - } - - def nextArrowColumnarBatch(): Any = { - val rb = resIter.next() - if (rb == null) { - val resultStructType = ArrowUtils.fromArrowSchema(outputSchema) - val resultColumnVectors = - ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray - return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0) - } - val outputNumRows = rb.getLength - val output = ConverterUtils.fromArrowRecordBatch(outputSchema, rb) - ConverterUtils.releaseArrowRecordBatch(rb) - val cb = new ColumnarBatch(output.map(v => v.asInstanceOf[ColumnVector]), outputNumRows) - val bytes: Long = cb 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) - cb - } - - override def next(): Any = { - if (!hasNext) { - throw new java.util.NoSuchElementException("End of stream") - } - if (!GlutenConfig.getConf.isClickHouseBackend) { - nextArrowColumnarBatch() - } else { - resIter.chNext() - } - - } - } - - // TODO: SPARK-25083 remove the type erasure hack in data source scan - new InterruptibleIterator(context, - ColumnarFactory.createClosableIterator(iter.asInstanceOf[Iterator[ColumnarBatch]])) + BackendsApiManager.getIteratorApiInstance.genFirstStageIterator(inputPartition, loadNative, + outputAttributes, context, jarList) } override def getPreferredLocations(split: Partition): Seq[String] = { diff --git a/jvm/src/main/scala/io/glutenproject/execution/NativeWholestageRowRDD.scala b/jvm/src/main/scala/io/glutenproject/execution/NativeWholestageRowRDD.scala index ba787ea15613..0878f3802a20 100644 --- a/jvm/src/main/scala/io/glutenproject/execution/NativeWholestageRowRDD.scala +++ b/jvm/src/main/scala/io/glutenproject/execution/NativeWholestageRowRDD.scala @@ -17,8 +17,6 @@ package io.glutenproject.execution -import java.io._ - import io.glutenproject.GlutenConfig import io.glutenproject.row.RowIterator import io.glutenproject.vectorized.ExpressionEvaluator @@ -50,8 +48,7 @@ class NativeWholestageRowRDD( } private def castNativePartition(split: Partition): BaseNativeFilePartition = split match { - case NativeSubstraitPartition(_, p: NativeFilePartition) => p - case NativeSubstraitPartition(_, m: NativeMergeTreePartition) => m + case NativeSubstraitPartition(_, p: BaseNativeFilePartition) => p case _ => throw new SparkException(s"[BUG] Not a NativeSubstraitPartition: $split") } @@ -63,7 +60,7 @@ class NativeWholestageRowRDD( var resIter : RowIterator = null if (loadNative) { val transKernel = new ExpressionEvaluator() - val inBatchIters = new java.util.ArrayList[ColumnarNativeIterator]() + val inBatchIters = new java.util.ArrayList[AbstractColumnarNativeIterator]() var startTime = System.nanoTime() resIter = transKernel.createKernelWithRowIterator(inputPartition.substraitPlan, inBatchIters) logWarning(s"===========create ${System.nanoTime() - startTime}") diff --git a/jvm/src/main/scala/io/glutenproject/execution/RowToArrowColumnarExec.scala b/jvm/src/main/scala/io/glutenproject/execution/RowToArrowColumnarExec.scala index 03ab53cdcb93..350bdfe919ed 100644 --- a/jvm/src/main/scala/io/glutenproject/execution/RowToArrowColumnarExec.scala +++ b/jvm/src/main/scala/io/glutenproject/execution/RowToArrowColumnarExec.scala @@ -19,211 +19,17 @@ package io.glutenproject.execution import java.util.concurrent.TimeUnit._ -import io.glutenproject.vectorized._ - import org.apache.spark.broadcast + 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.expressions.SortOrder -import org.apache.spark.sql.catalyst.expressions.SpecializedGetters +import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils.UnsafeItr -import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector -import org.apache.spark.sql.execution.vectorized.WritableColumnVector -import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch -class RowToColumnConverter(schema: StructType) extends Serializable { - private val converters = schema.fields.map { - f => RowToColumnConverter.getConverterForType(f.dataType, f.nullable) - } - - final def convert(row: InternalRow, vectors: Array[WritableColumnVector]): Unit = { - var idx = 0 - while (idx < row.numFields) { - converters(idx).append(row, idx, vectors(idx)) - idx += 1 - } - } -} - -object RowToColumnConverter { - private abstract class TypeConverter extends Serializable { - def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit - } - - private final case class BasicNullableTypeConverter(base: TypeConverter) extends TypeConverter { - override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { - if (row.isNullAt(column)) { - cv.appendNull - } else { - base.append(row, column, cv) - } - } - } - - private final case class StructNullableTypeConverter(base: TypeConverter) extends TypeConverter { - override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { - if (row.isNullAt(column)) { - cv.appendStruct(true) - } else { - base.append(row, column, cv) - } - } - } - - private def getConverterForType(dataType: DataType, nullable: Boolean): TypeConverter = { - val core = dataType match { - case BooleanType => BooleanConverter - case ByteType => ByteConverter - case ShortType => ShortConverter - case IntegerType | DateType => IntConverter - case FloatType => FloatConverter - case LongType | TimestampType => LongConverter - case DoubleType => DoubleConverter - case StringType => StringConverter - case BinaryType => BinaryConverter - case CalendarIntervalType => CalendarConverter - case at: ArrayType => new ArrayConverter(getConverterForType(at.elementType, nullable)) - case st: StructType => new StructConverter(st.fields.map( - (f) => getConverterForType(f.dataType, f.nullable))) - case dt: DecimalType => new DecimalConverter(dt) - case mt: MapType => new MapConverter(getConverterForType(mt.keyType, nullable), - getConverterForType(mt.valueType, nullable)) - case unknown => throw new UnsupportedOperationException( - s"Type $unknown not supported") - } - - if (nullable) { - dataType match { - case CalendarIntervalType => new StructNullableTypeConverter(core) - case st: StructType => new StructNullableTypeConverter(core) - case _ => new BasicNullableTypeConverter(core) - } - } else { - core - } - } - - private object BooleanConverter extends TypeConverter { - override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = - cv.appendBoolean(row.getBoolean(column)) - } - - private object ByteConverter extends TypeConverter { - override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = - cv.appendByte(row.getByte(column)) - } - - private object ShortConverter extends TypeConverter { - override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = - cv.appendShort(row.getShort(column)) - } - - private object IntConverter extends TypeConverter { - override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = - cv.appendInt(row.getInt(column)) - } - - private object FloatConverter extends TypeConverter { - override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = - cv.appendFloat(row.getFloat(column)) - } - - private object LongConverter extends TypeConverter { - override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = - cv.appendLong(row.getLong(column)) - } - - private object DoubleConverter extends TypeConverter { - override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = - cv.appendDouble(row.getDouble(column)) - } - - private object StringConverter extends TypeConverter { - override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { - val data = row.getUTF8String(column).getBytes - cv.asInstanceOf[ArrowWritableColumnVector].appendString(data, 0, data.length) - } - } - - private object BinaryConverter extends TypeConverter { - override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { - val data = row.getBinary(column) - cv.asInstanceOf[ArrowWritableColumnVector].appendString(data, 0, data.length) - } - } - - private object CalendarConverter extends TypeConverter { - override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { - val c = row.getInterval(column) - cv.appendStruct(false) - cv.getChild(0).appendInt(c.months) - cv.getChild(1).appendInt(c.days) - cv.getChild(2).appendLong(c.microseconds) - } - } - - private case class ArrayConverter(childConverter: TypeConverter) extends TypeConverter { - override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { - val values = row.getArray(column) - val numElements = values.numElements() - cv.appendArray(numElements) - val arrData = cv.arrayData() - for (i <- 0 until numElements) { - childConverter.append(values, i, arrData) - } - } - } - - private case class StructConverter(childConverters: Array[TypeConverter]) extends TypeConverter { - override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { - cv.appendStruct(false) - val data = row.getStruct(column, childConverters.length) - for (i <- 0 until childConverters.length) { - childConverters(i).append(data, i, cv.getChild(i)) - } - } - } - - private case class DecimalConverter(dt: DecimalType) extends TypeConverter { - override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { - val d = row.getDecimal(column, dt.precision, dt.scale) - if (dt.precision <= Decimal.MAX_INT_DIGITS) { - cv.appendInt(d.toUnscaledLong.toInt) - } else if (dt.precision <= Decimal.MAX_LONG_DIGITS) { - cv.appendLong(d.toUnscaledLong) - } else { - val value = d.toJavaBigDecimal - cv.asInstanceOf[ArrowWritableColumnVector].appendDecimal(value) - } - } - } - - private case class MapConverter(keyConverter: TypeConverter, valueConverter: TypeConverter) - extends TypeConverter { - override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { - val m = row.getMap(column) - val keys = cv.getChild(0) - val values = cv.getChild(1) - val numElements = m.numElements() - cv.appendArray(numElements) - - val srcKeys = m.keyArray() - val srcValues = m.valueArray() - - for (i <- 0 until numElements) { - keyConverter.append(srcKeys, i, keys) - valueConverter.append(srcValues, i, values) - } - } - } -} - /** * Provides a common executor to translate an [[RDD]] of [[InternalRow]] into an [[RDD]] of * [[ColumnarBatch]]. This is inserted whenever such a transition is determined to be needed. @@ -241,7 +47,7 @@ object RowToColumnConverter { * populate with [[RowToColumnConverter]], but the performance requirements are different and it * would only be to reduce code. */ -case class RowToArrowColumnarExec(child: SparkPlan) extends UnaryExecNode { +abstract class RowToArrowColumnarExec(child: SparkPlan) extends RowToColumnarExec(child = child) { override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning @@ -264,50 +70,9 @@ case class RowToArrowColumnarExec(child: SparkPlan) extends UnaryExecNode { "processTime" -> SQLMetrics.createTimingMetric(sparkContext, "totaltime_rowtoarrowcolumnar") ) - override def doExecuteColumnar(): RDD[ColumnarBatch] = { - val numInputRows = longMetric("numInputRows") - val numOutputBatches = longMetric("numOutputBatches") - val processTime = longMetric("processTime") - // Instead of creating a new config we are reusing columnBatchSize. In the future if we do - // combine with some of the Arrow conversion tools we will need to unify some of the configs. - val numRows = conf.columnBatchSize - // This avoids calling `schema` in the RDD closure, so that we don't need to include the entire - // plan (this) in the closure. - val localSchema = this.schema - child.execute().mapPartitions { rowIterator => - if (rowIterator.hasNext) { - val res = new Iterator[ColumnarBatch] { - private val converters = new RowToColumnConverter(localSchema) - private var last_cb: ColumnarBatch = null - private var elapse: Long = 0 - - override def hasNext: Boolean = { - rowIterator.hasNext - } + def doExecuteColumnarInternal(): RDD[ColumnarBatch] - override def next(): ColumnarBatch = { - val vectors: Seq[WritableColumnVector] = - ArrowWritableColumnVector.allocateColumns(numRows, schema) - var rowCount = 0 - while (rowCount < numRows && rowIterator.hasNext) { - val row = rowIterator.next() - val start = System.nanoTime() - converters.convert(row, vectors.toArray) - elapse += System.nanoTime() - start - rowCount += 1 - } - vectors.foreach(v => v.asInstanceOf[ArrowWritableColumnVector].setValueCount(rowCount)) - processTime.set(NANOSECONDS.toMillis(elapse)) - numInputRows += rowCount - numOutputBatches += 1 - last_cb = new ColumnarBatch(vectors.toArray, rowCount) - last_cb - } - } - new UnsafeItr(res) - } else { - Iterator.empty - } - } + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + doExecuteColumnarInternal() } } diff --git a/jvm/src/main/scala/io/glutenproject/execution/ShuffledHashJoinExecTransformer.scala b/jvm/src/main/scala/io/glutenproject/execution/ShuffledHashJoinExecTransformer.scala index 0f3b29216e3d..09f6014829a6 100644 --- a/jvm/src/main/scala/io/glutenproject/execution/ShuffledHashJoinExecTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/execution/ShuffledHashJoinExecTransformer.scala @@ -17,43 +17,19 @@ package io.glutenproject.execution -import java.util.concurrent.TimeUnit._ - -import io.glutenproject.vectorized._ -import io.glutenproject.GlutenConfig -import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD -import org.apache.spark.util.{ExecutorManager, UserAddedJarUtils, Utils} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.{BinaryExecNode, CodegenSupport, SparkPlan} +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.SQLMetrics -import scala.collection.JavaConverters._ -import org.apache.spark.internal.Logging -import org.apache.spark.sql.util.ArrowUtils -import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} - -import scala.collection.mutable.ListBuffer -import org.apache.arrow.vector.ipc.message.ArrowFieldNode -import org.apache.arrow.vector.ipc.message.ArrowRecordBatch -import org.apache.arrow.vector.types.pojo.ArrowType -import org.apache.arrow.vector.types.pojo.Field -import org.apache.arrow.vector.types.pojo.Schema -import org.apache.arrow.gandiva.expression._ -import org.apache.arrow.gandiva.evaluator._ -import org.apache.arrow.memory.ArrowBuf -import com.google.common.collect.Lists -import io.glutenproject.expression._ +import org.apache.spark.sql.vectorized.ColumnarBatch + import io.glutenproject.substrait.SubstraitContext -import io.glutenproject.vectorized.ExpressionEvaluator -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.{BaseJoinExec, HashJoin, ShuffledJoin} -import org.apache.spark.sql.execution.joins.HashedRelationInfo /** * Performs a hash join of two child relations by first shuffling the data using the join keys. diff --git a/jvm/src/main/scala/io/glutenproject/execution/SortExecTransformer.scala b/jvm/src/main/scala/io/glutenproject/execution/SortExecTransformer.scala index 57c98021075c..9bdc87527566 100644 --- a/jvm/src/main/scala/io/glutenproject/execution/SortExecTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/execution/SortExecTransformer.scala @@ -17,35 +17,14 @@ package io.glutenproject.execution -import io.glutenproject.GlutenConfig -import io.glutenproject.expression._ -import io.glutenproject.vectorized._ -import com.google.common.collect.Lists -import java.util.concurrent.TimeUnit._ - import io.glutenproject.substrait.SubstraitContext -import org.apache.arrow.vector.ipc.message.ArrowFieldNode -import org.apache.arrow.vector.ipc.message.ArrowRecordBatch -import org.apache.arrow.vector.types.pojo.ArrowType -import org.apache.arrow.vector.types.pojo.Field -import org.apache.arrow.vector.types.pojo.Schema -import org.apache.arrow.gandiva.expression._ -import org.apache.arrow.gandiva.evaluator._ -import org.apache.spark.{SparkContext, SparkEnv, TaskContext} -import org.apache.spark.executor.TaskMetrics import org.apache.spark.sql.execution._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.BoundReference -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.util.{ExecutorManager, UserAddedJarUtils, Utils} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} -import org.apache.spark.sql.types.DecimalType - -import scala.util.control.Breaks.{break, breakable} +import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.vectorized.ColumnarBatch case class SortExecTransformer( sortOrder: Seq[SortOrder], diff --git a/jvm/src/main/scala/io/glutenproject/execution/SortMergeJoinExecTransformer.scala b/jvm/src/main/scala/io/glutenproject/execution/SortMergeJoinExecTransformer.scala index 8e82d71408a5..be6db6c1c531 100644 --- a/jvm/src/main/scala/io/glutenproject/execution/SortMergeJoinExecTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/execution/SortMergeJoinExecTransformer.scala @@ -17,43 +17,16 @@ package io.glutenproject.execution -import java.util.concurrent.TimeUnit._ +import io.glutenproject.substrait.SubstraitContext -import io.glutenproject.vectorized._ -import io.glutenproject.GlutenConfig -import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD -import org.apache.spark.util.{UserAddedJarUtils, Utils} 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.catalyst.plans.physical._ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics - -import scala.collection.JavaConverters._ -import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.expressions.BoundReference -import org.apache.spark.sql.catalyst.expressions.AttributeReference -import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReference -import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} - -import scala.collection.mutable.ListBuffer -import org.apache.arrow.vector.ipc.message.ArrowFieldNode -import org.apache.arrow.vector.ipc.message.ArrowRecordBatch -import org.apache.arrow.vector.types.pojo.ArrowType -import org.apache.arrow.vector.types.pojo.Field -import org.apache.arrow.vector.types.pojo.Schema -import org.apache.arrow.gandiva.expression._ -import org.apache.arrow.gandiva.evaluator._ -import org.apache.arrow.memory.ArrowBuf -import com.google.common.collect.Lists -import io.glutenproject.expression._ -import io.glutenproject.substrait.SubstraitContext -import io.glutenproject.vectorized.ExpressionEvaluator -import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} -import org.apache.spark.sql.types.DecimalType +import org.apache.spark.sql.vectorized.ColumnarBatch /** * Performs a hash join of two child relations by first shuffling the data using the join keys. diff --git a/jvm/src/main/scala/io/glutenproject/execution/WholeStageTransformerExec.scala b/jvm/src/main/scala/io/glutenproject/execution/WholeStageTransformerExec.scala index 7fc1b332618f..535d0f672bf1 100644 --- a/jvm/src/main/scala/io/glutenproject/execution/WholeStageTransformerExec.scala +++ b/jvm/src/main/scala/io/glutenproject/execution/WholeStageTransformerExec.scala @@ -21,27 +21,22 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer import com.google.common.collect.Lists -import io.glutenproject.vectorized.ColumnarFactory import io.glutenproject.GlutenConfig +import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.expression._ import io.glutenproject.substrait.plan.{PlanBuilder, PlanNode} -import io.glutenproject.substrait.rel.{ExtensionTableBuilder, LocalFilesBuilder, RelNode} +import io.glutenproject.substrait.rel.RelNode import io.glutenproject.substrait.SubstraitContext import io.glutenproject.vectorized._ -import org.apache.arrow.gandiva.expression._ -import org.apache.arrow.vector.types.pojo.{ArrowType, Field} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils -import org.apache.spark.sql.execution.datasources.FilePartition import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.util.ArrowUtils -import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} -import org.apache.spark.util.{ExecutorManager, UserAddedJarUtils} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.UserAddedJarUtils case class TransformContext(inputAttributes: Seq[Attribute], outputAttributes: Seq[Attribute], root: RelNode) @@ -109,6 +104,7 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int override def supportsColumnar: Boolean = GlutenConfig.getConf.enableColumnarIterator override def otherCopyArgs: Seq[AnyRef] = Seq(transformStageId.asInstanceOf[Integer]) + override def generateTreeString( depth: Int, lastChildren: Seq[Boolean], @@ -133,6 +129,7 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int } override def nodeName: String = s"WholeStageCodegenTransformer (${transformStageId})" + def uploadAndListJars(signature: String): Seq[String] = if (signature != "") { if (sparkContext.listJars.filter(path => path.contains(s"${signature}.jar")).isEmpty) { @@ -166,7 +163,9 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int } } for (attr <- childCtx.outputAttributes) { - outNames.add(attr.name) + // ============================================ + // outNames.add(attr.name) + outNames.add(ConverterUtils.getShortAttributeName(attr) + "#" + attr.exprId.id) } val planNode = PlanBuilder.makePlan( substraitContext, Lists.newArrayList(childCtx.root), outNames) @@ -188,7 +187,8 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int override def updateMetrics(out_num_rows: Long, process_time: Long): Unit = {} var metricsUpdated: Boolean = false - def updateMetrics(nativeIterator: BatchIterator): Unit = { + + def updateMetrics(nativeIterator: AbstractBatchIterator): Unit = { if (metricsUpdated) return try { val metrics = nativeIterator.getMetrics @@ -212,47 +212,6 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int } } - def prepareRelationFunction( - keyAttributes: Seq[Attribute], - outputAttributes: Seq[Attribute]): TreeNode = { - val outputFieldList: List[Field] = outputAttributes.toList.map(attr => { - Field - .nullable(s"${attr.name.toUpperCase()}#${attr.exprId.id}", - CodeGeneration.getResultType(attr.dataType)) - }) - - val keyFieldList: List[Field] = keyAttributes.toList.map(attr => { - val field = Field - .nullable(s"${attr.name.toUpperCase()}#${attr.exprId.id}", - CodeGeneration.getResultType(attr.dataType)) - if (outputFieldList.indexOf(field) == -1) { - throw new UnsupportedOperationException(s"CachedRelation not found" + - s"${attr.name.toUpperCase()}#${attr.exprId.id} in ${outputAttributes}") - } - field - }) - - val key_args_node = TreeBuilder.makeFunction( - "key_field", - keyFieldList - .map(field => { - TreeBuilder.makeField(field) - }) - .asJava, - new ArrowType.Int(32, true) /*dummy ret type, won't be used*/ ) - - val cachedRelationFuncName = "CachedRelation" - val cached_relation_func = TreeBuilder.makeFunction( - cachedRelationFuncName, - Lists.newArrayList(key_args_node), - new ArrowType.Int(32, true) /*dummy ret type, won't be used*/ ) - - TreeBuilder.makeFunction( - "standalone", - Lists.newArrayList(cached_relation_func), - new ArrowType.Int(32, true)) - } - /** * Return built cpp library's signature */ @@ -300,37 +259,9 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int val wsCxt = doWholestageTransform() val startTime = System.nanoTime() - val substraitPlanPartition = fileScan.getPartitions.map { - case p: NativeMergeTreePartition => - val extensionTableNode = - ExtensionTableBuilder.makeExtensionTable(p.minParts, - p.maxParts, p.database, p.table, p.tablePath) - wsCxt.substraitContext.setExtensionTableNode(extensionTableNode) - // logWarning(s"The substrait plan for partition " + - // s"${p.index}:\n${wsCxt.root.toProtobuf.toString}") - p.copySubstraitPlan(wsCxt.root.toProtobuf.toByteArray) - case FilePartition(index, files) => - val paths = new java.util.ArrayList[String]() - val starts = new java.util.ArrayList[java.lang.Long]() - val lengths = new java.util.ArrayList[java.lang.Long]() - files.foreach { f => - paths.add(f.filePath) - starts.add(new java.lang.Long(f.start)) - lengths.add(new java.lang.Long(f.length)) - } - val localFilesNode = LocalFilesBuilder.makeLocalFiles(index, paths, starts, lengths) - wsCxt.substraitContext.setLocalFilesNode(localFilesNode) - val substraitPlan = wsCxt.root.toProtobuf - /* - val out = new DataOutputStream(new FileOutputStream("/tmp/SubStraitTest-Q6.dat", - false)); - out.write(substraitPlan.toByteArray()); - out.flush(); - */ - // logWarning(s"The substrait plan for partition ${index}:\n${substraitPlan.toString}") - NativeFilePartition(index, files, substraitPlan.toByteArray) - case p => p - } + val substraitPlanPartition = fileScan.getPartitions.map(p => + BackendsApiManager.getIteratorApiInstance.genNativeFilePartition(p, wsCxt) + ) logWarning( s"Generated substrait plan tooks: ${(System.nanoTime() - startTime) / 1000000} ms") @@ -348,17 +279,15 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int val numOutputBatches = child.longMetric("numOutputBatches") val pipelineTime = longMetric("pipelineTime") - var build_elapse: Long = 0 - var eval_elapse: Long = 0 // we should zip all dependent RDDs to current main RDD // TODO: Does it still need these parameters? val buildPlans = getBuildPlans val streamedSortPlan = getStreamedLeafPlan val dependentKernels: ListBuffer[ExpressionEvaluator] = ListBuffer() - val dependentKernelIterators: ListBuffer[BatchIterator] = ListBuffer() + val dependentKernelIterators: ListBuffer[AbstractBatchIterator] = ListBuffer() val buildRelationBatchHolder: ListBuffer[ColumnarBatch] = ListBuffer() - val serializableObjectHolder: ListBuffer[SerializableObject] = ListBuffer() - val relationHolder: ListBuffer[ColumnarHashedRelation] = ListBuffer() + // val serializableObjectHolder: ListBuffer[SerializableObject] = ListBuffer() + // val relationHolder: ListBuffer[ColumnarHashedRelation] = ListBuffer() // check if BatchScan exists val current_op = checkBatchScanExecTransformerChild() @@ -379,33 +308,9 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int val wsCxt = doWholestageTransform() val startTime = System.nanoTime() - val substraitPlanPartition = fileScan.getPartitions.map { - case p: NativeMergeTreePartition => - val extensionTableNode = - ExtensionTableBuilder.makeExtensionTable(p.minParts, - p.maxParts, p.database, p.table, p.tablePath) - wsCxt.substraitContext.setExtensionTableNode(extensionTableNode) -// logWarning(s"The substrait plan for partition " + -// s"${p.index}:\n${wsCxt.root.toProtobuf.toString}") - p.copySubstraitPlan(wsCxt.root.toProtobuf.toByteArray) - case FilePartition(index, files) => - val paths = new java.util.ArrayList[String]() - val starts = new java.util.ArrayList[java.lang.Long]() - val lengths = new java.util.ArrayList[java.lang.Long]() - files.foreach { f => - paths.add(f.filePath) - starts.add(new java.lang.Long(f.start)) - lengths.add(new java.lang.Long(f.length)) - } - - val localFilesNode = LocalFilesBuilder.makeLocalFiles(index, paths, starts, lengths) - wsCxt.substraitContext.setLocalFilesNode(localFilesNode) - val substraitPlan = wsCxt.root.toProtobuf - - logInfo(s"The substrait plan for partition ${index}:\n${substraitPlan.toString}") - NativeFilePartition(index, files, substraitPlan.toByteArray) - case p => p - } + val substraitPlanPartition = fileScan.getPartitions.map(p => + BackendsApiManager.getIteratorApiInstance.genNativeFilePartition(p, wsCxt) + ) logWarning( s"Generating the Substrait plan took: ${(System.nanoTime() - startTime) / 1000000} ms.") @@ -426,124 +331,15 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int val resCtx = doWholestageTransform() val outputAttributes = resCtx.outputAttributes val rootNode = resCtx.root - - if (!GlutenConfig.getConf.isClickHouseBackend) { - curRDD.mapPartitions { iter => - ExecutorManager.tryTaskSet(numaBindingInfo) - GlutenConfig.getConf - val execTempDir = GlutenConfig.getTempFile - val jarList = listJars.map(jarUrl => { - logWarning(s"Get Codegened library Jar ${jarUrl}") - UserAddedJarUtils.fetchJarFromSpark( - jarUrl, - execTempDir, - s"spark-columnar-plugin-codegen-precompile-${signature}.jar", - sparkConf) - s"${execTempDir}/spark-columnar-plugin-codegen-precompile-${signature}.jar" - }) - - val transKernel = new ExpressionEvaluator(jarList.toList.asJava) - val inBatchIter = new ColumnarNativeIterator(iter.asJava) - val inBatchIters = new java.util.ArrayList[ColumnarNativeIterator]() - inBatchIters.add(inBatchIter) - // we need to complete dependency RDD's firstly - val beforeBuild = System.nanoTime() - val outputSchema = ConverterUtils.toArrowSchema(outputAttributes) - val nativeIterator = transKernel.createKernelWithBatchIterator(rootNode, inBatchIters) - build_elapse += System.nanoTime() - beforeBuild - val resultStructType = ArrowUtils.fromArrowSchema(outputSchema) - val resIter = streamedSortPlan match { - case t: TransformSupport => - new Iterator[ColumnarBatch] { - override def hasNext: Boolean = { - val res = nativeIterator.hasNext - // if (res == false) updateMetrics(nativeIterator) - res - } - - override def next(): ColumnarBatch = { - val beforeEval = System.nanoTime() - val output_rb = nativeIterator.next - if (output_rb == null) { - eval_elapse += System.nanoTime() - beforeEval - val resultColumnVectors = - ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray - return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0) - } - val outputNumRows = output_rb.getLength - val outSchema = ConverterUtils.toArrowSchema(resCtx.outputAttributes) - val output = ConverterUtils.fromArrowRecordBatch(outSchema, output_rb) - ConverterUtils.releaseArrowRecordBatch(output_rb) - eval_elapse += System.nanoTime() - beforeEval - new ColumnarBatch(output.map(v => v.asInstanceOf[ColumnVector]), outputNumRows) - } - } - case _ => - throw new UnsupportedOperationException( - s"streamedSortPlan should support transformation") - } - var closed = false - - def close = { - closed = true - pipelineTime += (eval_elapse + build_elapse) / 1000000 - buildRelationBatchHolder.foreach(_.close) // fixing: ref cnt goes nagative - dependentKernels.foreach(_.close) - dependentKernelIterators.foreach(_.close) - nativeIterator.close() - relationHolder.clear() - } - - SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit](_ => { - close - }) - ColumnarFactory.createClosableIterator(resIter) - } - } else { - curRDD.mapPartitions { iter => - GlutenConfig.getConf - val transKernel = new ExpressionEvaluator() - val inBatchIter = new CHColumnarNativeIterator(iter.asJava) - val inBatchIters = new java.util.ArrayList[ColumnarNativeIterator]() - inBatchIters.add(inBatchIter) - // we need to complete dependency RDD's firstly - val beforeBuild = System.nanoTime() - val nativeIterator = transKernel.createKernelWithBatchIterator(rootNode, inBatchIters) - build_elapse += System.nanoTime() - beforeBuild - val resIter = streamedSortPlan match { - case t: TransformSupport => - new Iterator[ColumnarBatch] { - override def hasNext: Boolean = { - val res = nativeIterator.hasNext - res - } - - override def next(): ColumnarBatch = { - val beforeEval = System.nanoTime() - nativeIterator.chNext() - } - } - case _ => - throw new UnsupportedOperationException( - s"streamedSortPlan should support transformation") - } - var closed = false - - def close = { - closed = true - pipelineTime += (eval_elapse + build_elapse) / 1000000 - buildRelationBatchHolder.foreach(_.close) // fixing: ref cnt goes nagative - dependentKernels.foreach(_.close) - dependentKernelIterators.foreach(_.close) - nativeIterator.close() - relationHolder.clear() - } - - SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit](_ => { - close - }) - ColumnarFactory.createClosableIterator(resIter) - } + // logInfo(s"The substrait plan for final agg " + + // s"\n${rootNode.toProtobuf.toString}") + + curRDD.mapPartitions { iter => + BackendsApiManager.getIteratorApiInstance.genFinalStageIterator(iter, numaBindingInfo, + listJars, signature, sparkConf, outputAttributes, + rootNode, streamedSortPlan, + pipelineTime, buildRelationBatchHolder, + dependentKernels, dependentKernelIterators) } } } diff --git a/jvm/src/main/scala/io/glutenproject/execution/WindowExecTransformer.scala b/jvm/src/main/scala/io/glutenproject/execution/WindowExecTransformer.scala index cb2c79474852..b9f88e6834da 100644 --- a/jvm/src/main/scala/io/glutenproject/execution/WindowExecTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/execution/WindowExecTransformer.scala @@ -17,38 +17,16 @@ package io.glutenproject.execution -import java.util.concurrent.TimeUnit - -import com.google.flatbuffers.FlatBufferBuilder -import io.glutenproject.GlutenConfig -import io.glutenproject.expression.{CodeGeneration, ConverterUtils} import io.glutenproject.substrait.SubstraitContext -import io.glutenproject.vectorized.{ArrowWritableColumnVector, CloseableColumnBatchIterator, ExpressionEvaluator} -import org.apache.arrow.gandiva.expression.TreeBuilder -import org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID -import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema} -import org.apache.spark.internal.Logging + import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, Attribute, AttributeReference, Cast, Descending, Expression, Literal, MakeDecimal, NamedExpression, PredicateHelper, Rank, SortOrder, UnscaledValue, WindowExpression, WindowFunction, WindowSpecDefinition} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, NamedExpression, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} -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.SparkPlan import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.window.WindowExecBase -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{DataType, DateType, DecimalType, DoubleType, IntegerType, LongType, StringType, TimestampType} -import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.ExecutorManager - -import scala.collection.JavaConverters._ -import scala.collection.immutable.Stream.Empty -import scala.collection.mutable.ListBuffer -import scala.util.Random -import util.control.Breaks._ case class WindowExecTransformer(windowExpression: Seq[NamedExpression], partitionSpec: Seq[Expression], @@ -103,7 +81,7 @@ case class WindowExecTransformer(windowExpression: Seq[NamedExpression], override def hashCode(): Int = System.identityHashCode(this) - private object NoneType { + /* private object NoneType { val NONE_TYPE = new NoneType } @@ -125,7 +103,7 @@ case class WindowExecTransformer(windowExpression: Seq[NamedExpression], } override def isComplex: Boolean = false - } + } */ override protected def doExecute(): RDD[InternalRow] = { throw new UnsupportedOperationException() diff --git a/jvm/src/main/scala/io/glutenproject/expression/BinaryExpressionTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/BinaryExpressionTransformer.scala index 7d8140bb1516..750c20ca5d88 100644 --- a/jvm/src/main/scala/io/glutenproject/expression/BinaryExpressionTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/expression/BinaryExpressionTransformer.scala @@ -20,9 +20,7 @@ package io.glutenproject.expression import com.google.common.collect.Lists import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types._ -import scala.collection.mutable.ListBuffer import io.glutenproject.expression.DateTimeExpressionsTransformer.{DateDiffTransformer, UnixTimestampTransformer} import io.glutenproject.substrait.`type`.TypeBuilder import io.glutenproject.substrait.expression.{ExpressionBuilder, ExpressionNode} diff --git a/jvm/src/main/scala/io/glutenproject/expression/BoundReferenceTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/BoundReferenceTransformer.scala index 9be7e3883279..c5d1000bcd26 100644 --- a/jvm/src/main/scala/io/glutenproject/expression/BoundReferenceTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/expression/BoundReferenceTransformer.scala @@ -17,10 +17,9 @@ package io.glutenproject.expression -import com.google.common.collect.Lists import io.glutenproject.substrait.expression.{ExpressionBuilder, ExpressionNode} -import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.internal.Logging import org.apache.spark.sql.types._ class BoundReferenceTransformer(ordinal: Int, dataType: DataType, nullable: Boolean) diff --git a/jvm/src/main/scala/io/glutenproject/expression/CaseWhenOperatorTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/CaseWhenOperatorTransformer.scala index 2adf3c47021b..236fe89bf311 100644 --- a/jvm/src/main/scala/io/glutenproject/expression/CaseWhenOperatorTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/expression/CaseWhenOperatorTransformer.scala @@ -17,14 +17,9 @@ package io.glutenproject.expression -import com.google.common.collect.Lists -import com.google.common.collect.Sets import io.glutenproject.substrait.expression.ExpressionNode import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types._ - -import scala.collection.mutable.ListBuffer /** * A version of substring that supports columnar processing for utf8. diff --git a/jvm/src/main/scala/io/glutenproject/expression/CoalesceOperatorTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/CoalesceOperatorTransformer.scala index ed4a705c7582..10bc9dd4d384 100644 --- a/jvm/src/main/scala/io/glutenproject/expression/CoalesceOperatorTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/expression/CoalesceOperatorTransformer.scala @@ -17,14 +17,9 @@ package io.glutenproject.expression -import com.google.common.collect.Lists -import com.google.common.collect.Sets import io.glutenproject.substrait.expression.ExpressionNode import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types._ - -import scala.collection.mutable.ListBuffer /** * An expression that is evaluated to the first non-null input. diff --git a/jvm/src/main/scala/io/glutenproject/expression/ConcatOperatorTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/ConcatOperatorTransformer.scala index bfd9d061e5a7..c002fafdc2fc 100644 --- a/jvm/src/main/scala/io/glutenproject/expression/ConcatOperatorTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/expression/ConcatOperatorTransformer.scala @@ -16,14 +16,9 @@ */ package io.glutenproject.expression -import com.google.common.collect.Lists -import com.google.common.collect.Sets import io.glutenproject.substrait.expression.ExpressionNode import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types._ - -import scala.collection.mutable.ListBuffer class ConcatTransformer(exps: Seq[Expression], original: Expression) extends Concat(exps: Seq[Expression]) diff --git a/jvm/src/main/scala/io/glutenproject/expression/ConverterUtils.scala b/jvm/src/main/scala/io/glutenproject/expression/ConverterUtils.scala index e09b36f37555..2c5225871621 100644 --- a/jvm/src/main/scala/io/glutenproject/expression/ConverterUtils.scala +++ b/jvm/src/main/scala/io/glutenproject/expression/ConverterUtils.scala @@ -17,287 +17,16 @@ package io.glutenproject.expression -import java.io._ -import java.nio.channels.Channels - -import scala.collection.JavaConverters._ - -import com.google.common.collect.Lists import io.glutenproject.substrait.`type`._ -import io.glutenproject.vectorized.ArrowWritableColumnVector -import io.netty.buffer.{ByteBufAllocator, ByteBufOutputStream} -import org.apache.arrow.flatbuf.MessageHeader -import org.apache.arrow.gandiva.evaluator._ -import org.apache.arrow.gandiva.exceptions.GandivaException -import org.apache.arrow.gandiva.expression._ -import org.apache.arrow.gandiva.ipc.GandivaTypes -import org.apache.arrow.gandiva.ipc.GandivaTypes.ExpressionList -import org.apache.arrow.memory.BufferAllocator -import org.apache.arrow.vector._ -import org.apache.arrow.vector.ipc.{ReadChannel, WriteChannel} -import org.apache.arrow.vector.ipc.message._ -import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema} -import org.apache.arrow.vector.types.TimeUnit -import org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.optimizer._ -import org.apache.spark.sql.catalyst.util.DateTimeConstants -import org.apache.spark.sql.execution.datasources.v2.arrow.{SparkSchemaUtils, SparkVectorUtils} import org.apache.spark.sql.types._ -import org.apache.spark.sql.util.ArrowUtils -import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.sql.vectorized.ColumnarBatch object ConverterUtils extends Logging { - def calcuateEstimatedSize(columnarBatch: ColumnarBatch): Long = { - SparkVectorUtils.estimateSize(columnarBatch) - } - - def createArrowRecordBatch(columnarBatch: ColumnarBatch): ArrowRecordBatch = { - SparkVectorUtils.toArrowRecordBatch(columnarBatch) - } - - def createArrowRecordBatch(numRowsInBatch: Int, cols: List[ValueVector]): ArrowRecordBatch = { - SparkVectorUtils.toArrowRecordBatch(numRowsInBatch, cols) - } - - def convertToNetty(iter: Array[ColumnarBatch], out: OutputStream): Unit = { - val channel = new WriteChannel(Channels.newChannel(out)) - var schema: Schema = null - val option = new IpcOption - - iter.foreach { columnarBatch => - val vectors = (0 until columnarBatch.numCols) - .map(i => columnarBatch.column(i).asInstanceOf[ArrowWritableColumnVector]) - .toList - try { - if (schema == null) { - schema = new Schema(vectors.map(_.getValueVector().getField).asJava) - MessageSerializer.serialize(channel, schema, option) - } - val batch = ConverterUtils - .createArrowRecordBatch(columnarBatch.numRows, vectors.map(_.getValueVector)) - try { - MessageSerializer.serialize(channel, batch, option) - } finally { - batch.close() - } - } catch { - case e => - System.err.println(s"conversion failed") - e.printStackTrace() - throw e - } - } - } - - def convertToNetty(iter: Array[ColumnarBatch]): Array[Byte] = { - val innerBuf = ByteBufAllocator.DEFAULT.buffer() - val outStream = new ByteBufOutputStream(innerBuf) - convertToNetty(iter, outStream) - val bytes = new Array[Byte](innerBuf.readableBytes); - innerBuf.getBytes(innerBuf.readerIndex, bytes); - innerBuf.release() - outStream.close() - bytes - } - - def convertFromNetty( - attributes: Seq[Attribute], - input: InputStream): Iterator[ColumnarBatch] = { - new Iterator[ColumnarBatch] { - val allocator = ArrowWritableColumnVector.getOffRecordAllocator - var messageReader = - new MessageChannelReader(new ReadChannel(Channels.newChannel(input)), allocator) - var schema: Schema = null - var result: MessageResult = null - - override def hasNext: Boolean = - if (input.available > 0) { - return true - } else { - messageReader.close - return false - } - override def next(): ColumnarBatch = { - if (input.available == 0) { - if (attributes == null) { - return null - } - val resultStructType = StructType( - attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) - val resultColumnVectors = - ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray - return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0) - } - try { - if (schema == null) { - result = messageReader.readNext(); - if (result == null) { - throw new IOException("Unexpected end of input. Missing schema."); - } - if (result.getMessage().headerType() != MessageHeader.Schema) { - throw new IOException( - "Expected schema but header was " + result.getMessage().headerType()); - } - schema = MessageSerializer.deserializeSchema(result.getMessage()); - } - result = messageReader.readNext(); - if (result.getMessage().headerType() != MessageHeader.RecordBatch) { - throw new IOException( - "Expected recordbatch but header was " + result.getMessage().headerType()); - } - var bodyBuffer = result.getBodyBuffer(); - - // For zero-length batches, need an empty buffer to deserialize the batch - if (bodyBuffer == null) { - bodyBuffer = allocator.getEmpty(); - } - - val batch = MessageSerializer.deserializeRecordBatch(result.getMessage(), bodyBuffer); - val vectors = fromArrowRecordBatch(schema, batch, allocator) - val length = batch.getLength - batch.close - new ColumnarBatch(vectors.map(_.asInstanceOf[ColumnVector]), length) - } catch { - case e: Throwable => - messageReader.close - throw e - } - } - } - } - - def convertFromNetty( - attributes: Seq[Attribute], - data: Array[Array[Byte]], - columnIndices: Array[Int] = null): Iterator[ColumnarBatch] = { - if (data.size == 0) { - return new Iterator[ColumnarBatch] { - override def hasNext: Boolean = false - override def next(): ColumnarBatch = { - val resultStructType = if (columnIndices == null) { - StructType( - attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) - } else { - StructType( - columnIndices - .map(i => attributes(i)) - .map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) - } - val resultColumnVectors = - ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray - return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0) - } - } - } - new Iterator[ColumnarBatch] { - var array_id = 0 - val allocator = ArrowWritableColumnVector.getOffRecordAllocator - var input = new ByteArrayInputStream(data(array_id)) - var messageReader = - new MessageChannelReader(new ReadChannel(Channels.newChannel(input)), allocator) - var schema: Schema = null - var result: MessageResult = null - - override def hasNext: Boolean = - if (array_id < (data.size - 1) || input.available > 0) { - return true - } else { - messageReader.close - return false - } - override def next(): ColumnarBatch = { - if (input.available == 0) { - messageReader.close - array_id += 1 - input = new ByteArrayInputStream(data(array_id)) - messageReader = - new MessageChannelReader(new ReadChannel(Channels.newChannel(input)), allocator) - } - if (input.available == 0) { - val resultStructType = StructType( - attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) - val resultColumnVectors = - ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray - return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0) - } - try { - if (schema == null) { - result = messageReader.readNext(); - - if (result == null) { - throw new IOException("Unexpected end of input. Missing schema."); - } - - if (result.getMessage().headerType() != MessageHeader.Schema) { - throw new IOException( - "Expected schema but header was " + result.getMessage().headerType()); - } - - schema = MessageSerializer.deserializeSchema(result.getMessage()); - - } - - result = messageReader.readNext(); - if (result.getMessage().headerType() == MessageHeader.Schema) { - result = messageReader.readNext(); - } - - if (result.getMessage().headerType() != MessageHeader.RecordBatch) { - throw new IOException( - "Expected recordbatch but header was " + result.getMessage().headerType()); - } - var bodyBuffer = result.getBodyBuffer(); - - // For zero-length batches, need an empty buffer to deserialize the batch - if (bodyBuffer == null) { - bodyBuffer = allocator.getEmpty(); - } - - val batch = MessageSerializer.deserializeRecordBatch(result.getMessage(), bodyBuffer); - val vectors = fromArrowRecordBatch(schema, batch, allocator) - val length = batch.getLength - batch.close - if (columnIndices == null) { - new ColumnarBatch(vectors.map(_.asInstanceOf[ColumnVector]), length) - } else { - new ColumnarBatch( - columnIndices.map(i => vectors(i).asInstanceOf[ColumnVector]), - length) - } - - } catch { - case e: Throwable => - messageReader.close - throw e - } - } - } - } - - def fromArrowRecordBatch( - recordBatchSchema: Schema, - recordBatch: ArrowRecordBatch, - allocator: BufferAllocator = null): Array[ArrowWritableColumnVector] = { - val numRows = recordBatch.getLength() - ArrowWritableColumnVector.loadColumns(numRows, recordBatchSchema, recordBatch, allocator) - } - - def releaseArrowRecordBatch(recordBatch: ArrowRecordBatch): Unit = { - if (recordBatch != null) { - recordBatch.close() - } - } - - def releaseArrowRecordBatchList(recordBatchList: Array[ArrowRecordBatch]): Unit = { - recordBatchList.foreach({ recordBatch => - if (recordBatch != null) - releaseArrowRecordBatch(recordBatch) - }) - } def getAttrFromExpr(fieldExpr: Expression, skipAlias: Boolean = false): AttributeReference = { fieldExpr match { @@ -340,6 +69,15 @@ object ConverterUtils extends Logging { } } + def getShortAttributeName(attr: Attribute): String = { + val subIndex = attr.name.indexOf("(") + if (subIndex != -1) { + attr.name.substring(0, subIndex) + } else { + attr.name + } + } + def getResultAttrFromExpr( fieldExpr: Expression, name: String = "None", @@ -426,190 +164,10 @@ object ConverterUtils extends Logging { true } - def combineArrowRecordBatch(rb1: ArrowRecordBatch, rb2: ArrowRecordBatch): ArrowRecordBatch = { - val numRows = rb1.getLength() - val rb1_nodes = rb1.getNodes() - val rb2_nodes = rb2.getNodes() - val rb1_bufferlist = rb1.getBuffers() - val rb2_bufferlist = rb2.getBuffers() - - val combined_nodes = rb1_nodes.addAll(rb2_nodes) - val combined_bufferlist = rb1_bufferlist.addAll(rb2_bufferlist) - new ArrowRecordBatch(numRows, rb1_nodes, rb1_bufferlist) - } - - def toArrowSchema(attributes: Seq[Attribute]): Schema = { - val fields = attributes.map(attr => { - Field - .nullable(s"${attr.name}#${attr.exprId.id}", CodeGeneration.getResultType(attr.dataType)) - }) - new Schema(fields.toList.asJava) - } - - def toArrowSchema(schema: StructType): Schema = { - val fields = schema - .map(field => { - Field.nullable(field.name, CodeGeneration.getResultType(field.dataType)) - }) - new Schema(fields.toList.asJava) - } - override def toString(): String = { s"ConverterUtils" } - @throws[IOException] - def getSchemaBytesBuf(schema: Schema): Array[Byte] = { - val out: ByteArrayOutputStream = new ByteArrayOutputStream - MessageSerializer.serialize(new WriteChannel(Channels.newChannel(out)), schema) - out.toByteArray - } - - @throws[IOException] - def getSchemaFromBytesBuf(schema: Array[Byte]): Schema = { - val in: ByteArrayInputStream = new ByteArrayInputStream(schema) - MessageSerializer.deserializeSchema(new ReadChannel(Channels.newChannel(in))) - } - - @throws[GandivaException] - def getExprListBytesBuf(exprs: List[ExpressionTree]): Array[Byte] = { - val builder: ExpressionList.Builder = GandivaTypes.ExpressionList.newBuilder - exprs.foreach { expr => builder.addExprs(expr.toProtobuf) } - builder.build.toByteArray - } - - def checkIfTypeSupported(dt: DataType): Unit = dt match { - case d: BooleanType => - case d: ByteType => - case d: ShortType => - case d: IntegerType => - case d: LongType => - case d: FloatType => - case d: DoubleType => - case d: StringType => - case d: DateType => - case d: DecimalType => - case d: TimestampType => - case _ => - throw new UnsupportedOperationException(s"Unsupported data type: $dt") - } - - def createArrowField(name: String, dt: DataType): Field = dt match { - case at: ArrayType => - new Field( - name, - FieldType.nullable(ArrowType.List.INSTANCE), - Lists.newArrayList(createArrowField(s"${name}_${dt}", at.elementType))) - case mt: MapType => - throw new UnsupportedOperationException(s"${dt} is not supported yet") - case st: StructType => - throw new UnsupportedOperationException(s"${dt} is not supported yet") - case _ => - Field.nullable(name, CodeGeneration.getResultType(dt)) - } - - def createArrowField(attr: Attribute): Field = - createArrowField(s"${attr.name}#${attr.exprId.id}", attr.dataType) - - private def asTimestampType(inType: ArrowType): ArrowType.Timestamp = { - if (inType.getTypeID != ArrowTypeID.Timestamp) { - throw new IllegalArgumentException(s"Value type to convert must be timestamp") - } - inType.asInstanceOf[ArrowType.Timestamp] - } - - def convertTimestampZone(inNode: TreeNode, inType: ArrowType, - toZone: String): (TreeNode, ArrowType) = { - throw new UnsupportedOperationException("not implemented") // fixme 20210602 hongze - val inTimestamp = asTimestampType(inType) - val fromZone = inTimestamp.getTimezone - - val (outNode0: TreeNode, outTimestamp0: ArrowType.Timestamp) = - if (SparkSchemaUtils.timeZoneIDEquals(fromZone, toZone)) { - val outType = new ArrowType.Timestamp(inTimestamp.getUnit, toZone) - (inNode, outType) - } else { - // todo conversion - } - (outNode0, outTimestamp0) - } - - def convertTimestampToMilli(inNode: TreeNode, inType: ArrowType): (TreeNode, ArrowType) = { - val inTimestamp = asTimestampType(inType) - inTimestamp.getUnit match { - case TimeUnit.MILLISECOND => (inNode, inType) - case TimeUnit.MICROSECOND => - // truncate from micro to milli - val outType = new ArrowType.Timestamp(TimeUnit.MILLISECOND, - inTimestamp.getTimezone) - (TreeBuilder.makeFunction( - "convertTimestampUnit", - Lists.newArrayList(inNode), outType), outType) - } - } - - def convertTimestampToMicro(inNode: TreeNode, inType: ArrowType): (TreeNode, ArrowType) = { - val inTimestamp = asTimestampType(inType) - inTimestamp.getUnit match { - case TimeUnit.MICROSECOND => (inNode, inType) - case TimeUnit.MILLISECOND => - // truncate from micro to milli - val outType = new ArrowType.Timestamp(TimeUnit.MICROSECOND, - inTimestamp.getTimezone) - (TreeBuilder.makeFunction( - "convertTimestampUnit", - Lists.newArrayList(inNode), outType), outType) - } - } - - def toInt32(inNode: TreeNode, inType: ArrowType): (TreeNode, ArrowType) = { - val toType = ArrowUtils.toArrowType(IntegerType, null) - val toNode = TreeBuilder.makeFunction("castINT", Lists.newArrayList(inNode), - toType) - (toNode, toType) - } - - // use this carefully - def toGandivaMicroUTCTimestamp(inNode: TreeNode, inType: ArrowType, - timeZoneId: Option[String] = None): (TreeNode, ArrowType) = { - val zoneId = timeZoneId.orElse(Some(SparkSchemaUtils.getLocalTimezoneID())).get - val utcTimestampNodeOriginal = inNode - val inTimestampType = asTimestampType(inType) - val inTimestampTypeUTC = new ArrowType.Timestamp(inTimestampType.getUnit, - "UTC") - ConverterUtils.convertTimestampToMicro(utcTimestampNodeOriginal, - inTimestampTypeUTC) - } - - // use this carefully - def toGandivaTimestamp(inNode: TreeNode, inType: ArrowType, - timeZoneId: Option[String] = None): (TreeNode, ArrowType) = { - val zoneId = timeZoneId.orElse(Some(SparkSchemaUtils.getLocalTimezoneID())).get - - val utcTimestampNodeOriginal = inNode - val utcTimestampNodeMilli = ConverterUtils.convertTimestampToMilli(utcTimestampNodeOriginal, - inType)._1 - val utcTimestampNodeLong = TreeBuilder.makeFunction("castBIGINT", - Lists.newArrayList(utcTimestampNodeMilli), new ArrowType.Int(64, - true)) - val diff = SparkSchemaUtils.getTimeZoneIDOffset(zoneId) * - DateTimeConstants.MILLIS_PER_SECOND - - val localizedTimestampNodeLong = TreeBuilder.makeFunction("add", - Lists.newArrayList(utcTimestampNodeLong, - TreeBuilder.makeLiteral(java.lang.Long.valueOf(diff))), - new ArrowType.Int(64, true)) - val localized = new ArrowType.Timestamp(TimeUnit.MILLISECOND, null) - val localizedTimestampNode = TreeBuilder.makeFunction("castTIMESTAMP", - Lists.newArrayList(localizedTimestampNodeLong), localized) - (localizedTimestampNode, localized) - } - - def toSparkTimestamp(inNode: TreeNode, inType: ArrowType, - timeZoneId: Option[String] = None): (TreeNode, ArrowType) = { - throw new UnsupportedOperationException() - } - def powerOfTen(pow: Int): (String, Int, Int) = { val POWERS_OF_10: Array[(String, Int, Int)] = Array( ("1", 1, 0), diff --git a/jvm/src/main/scala/io/glutenproject/expression/DateTimeExpressionsTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/DateTimeExpressionsTransformer.scala index d06269721989..25e4be906844 100644 --- a/jvm/src/main/scala/io/glutenproject/expression/DateTimeExpressionsTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/expression/DateTimeExpressionsTransformer.scala @@ -17,42 +17,11 @@ package io.glutenproject.expression -import java.util.Collections - -import com.google.common.collect.Lists -import io.glutenproject.expression.DateTimeExpressionsTransformer.castDateFromTimestamp import io.glutenproject.substrait.expression.ExpressionNode -import org.apache.arrow.gandiva.expression.TreeBuilder -import org.apache.arrow.gandiva.expression.TreeNode -import org.apache.arrow.vector.types.DateUnit -import org.apache.arrow.vector.types.pojo.ArrowType -import org.apache.spark.sql.catalyst.expressions.CheckOverflow -import org.apache.spark.sql.catalyst.expressions.CurrentDate -import org.apache.spark.sql.catalyst.expressions.CurrentTimestamp -import org.apache.spark.sql.catalyst.expressions.DateDiff -import org.apache.spark.sql.catalyst.expressions.DayOfMonth -import org.apache.spark.sql.catalyst.expressions.DayOfWeek -import org.apache.spark.sql.catalyst.expressions.DayOfYear -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.expressions.Hour -import org.apache.spark.sql.catalyst.expressions.MakeDate -import org.apache.spark.sql.catalyst.expressions.MakeTimestamp -import org.apache.spark.sql.catalyst.expressions.MicrosToTimestamp -import org.apache.spark.sql.catalyst.expressions.MillisToTimestamp -import org.apache.spark.sql.catalyst.expressions.Minute -import org.apache.spark.sql.catalyst.expressions.Month -import org.apache.spark.sql.catalyst.expressions.Now -import org.apache.spark.sql.catalyst.expressions.Second -import org.apache.spark.sql.catalyst.expressions.SecondsToTimestamp -import org.apache.spark.sql.catalyst.expressions.UnixDate -import org.apache.spark.sql.catalyst.expressions.UnixMicros -import org.apache.spark.sql.catalyst.expressions.UnixMillis -import org.apache.spark.sql.catalyst.expressions.UnixSeconds -import org.apache.spark.sql.catalyst.expressions.UnixTimestamp -import org.apache.spark.sql.catalyst.expressions.Year + +import org.apache.spark.sql.catalyst.expressions.{CurrentDate, CurrentTimestamp, DateDiff, DayOfMonth, DayOfWeek, DayOfYear, Expression, Hour, MakeDate, MakeTimestamp, MicrosToTimestamp, MillisToTimestamp, Minute, Month, Now, Second, SecondsToTimestamp, UnixDate, UnixMicros, UnixMillis, UnixSeconds, UnixTimestamp, Year} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{DateType, IntegerType, LongType, StringType, TimestampType} -import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.types.{DateType, TimestampType} object DateTimeExpressionsTransformer { class CurrentTimestampTransformer() extends CurrentTimestamp with ExpressionTransformer { diff --git a/jvm/src/main/scala/io/glutenproject/expression/IfOperatorTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/IfOperatorTransformer.scala index 1ef520613b5a..6903ea0dae5c 100644 --- a/jvm/src/main/scala/io/glutenproject/expression/IfOperatorTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/expression/IfOperatorTransformer.scala @@ -17,12 +17,9 @@ package io.glutenproject.expression -import com.google.common.collect.Lists -import com.google.common.collect.Sets import io.glutenproject.substrait.expression.ExpressionNode import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types._ class IfTransformer(predicate: Expression, trueValue: Expression, falseValue: Expression, original: Expression) diff --git a/jvm/src/main/scala/io/glutenproject/expression/InOperatorTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/InOperatorTransformer.scala index b2dff7ce29a4..74378fa584c6 100644 --- a/jvm/src/main/scala/io/glutenproject/expression/InOperatorTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/expression/InOperatorTransformer.scala @@ -17,14 +17,9 @@ package io.glutenproject.expression -import com.google.common.collect.Lists -import com.google.common.collect.Sets import io.glutenproject.substrait.expression.ExpressionNode import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types._ - -import scala.collection.mutable.ListBuffer class InTransformer(value: Expression, list: Seq[Expression], original: Expression) extends In(value: Expression, list: Seq[Expression]) diff --git a/jvm/src/main/scala/io/glutenproject/expression/InSetOperatorTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/InSetOperatorTransformer.scala index 5ca8a8702c5d..73459a120cba 100644 --- a/jvm/src/main/scala/io/glutenproject/expression/InSetOperatorTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/expression/InSetOperatorTransformer.scala @@ -17,17 +17,9 @@ package io.glutenproject.expression -import java.lang - -import com.google.common.collect.Lists -import com.google.common.collect.Sets import io.glutenproject.substrait.expression.ExpressionNode import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String - -import scala.collection.mutable.ListBuffer class InSetTransformer(value: Expression, hset: Set[Any], original: Expression) extends InSet(value: Expression, hset: Set[Any]) diff --git a/jvm/src/main/scala/io/glutenproject/expression/RoundOperatorTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/RoundOperatorTransformer.scala index b51ffd16377b..c2e2aff6a79e 100644 --- a/jvm/src/main/scala/io/glutenproject/expression/RoundOperatorTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/expression/RoundOperatorTransformer.scala @@ -17,14 +17,9 @@ package io.glutenproject.expression -import com.google.common.collect.Lists import io.glutenproject.substrait.expression.ExpressionNode import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.optimizer._ -import org.apache.spark.sql.types._ - -import scala.collection.mutable.ListBuffer class RoundExpression(child: Expression, scale: Expression, original: Expression) extends Round(child: Expression, scale: Expression) diff --git a/jvm/src/main/scala/io/glutenproject/expression/ScalarSubqueryTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/ScalarSubqueryTransformer.scala index d277bc875676..091c80c662bd 100644 --- a/jvm/src/main/scala/io/glutenproject/expression/ScalarSubqueryTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/expression/ScalarSubqueryTransformer.scala @@ -17,25 +17,13 @@ package io.glutenproject.expression -import com.google.common.collect.Lists import io.glutenproject.substrait.expression.ExpressionNode -import org.apache.arrow.gandiva.evaluator._ -import org.apache.arrow.gandiva.exceptions.GandivaException -import org.apache.arrow.gandiva.expression._ -import org.apache.arrow.vector.types.DateUnit -import org.apache.arrow.vector.types.pojo.ArrowType -import org.apache.arrow.vector.types.pojo.Field -import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.{InternalRow, expressions} +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} -import org.apache.spark.sql.execution.BaseSubqueryExec -import org.apache.spark.sql.execution.ExecSubqueryExpression import org.apache.spark.sql.execution.ScalarSubquery import org.apache.spark.sql.types._ -import scala.collection.mutable.ListBuffer - class ScalarSubqueryTransformer( query: ScalarSubquery) extends Expression with ExpressionTransformer { diff --git a/jvm/src/main/scala/io/glutenproject/expression/TernaryOperatorTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/TernaryOperatorTransformer.scala index 9278642da3b0..6bb019715e05 100644 --- a/jvm/src/main/scala/io/glutenproject/expression/TernaryOperatorTransformer.scala +++ b/jvm/src/main/scala/io/glutenproject/expression/TernaryOperatorTransformer.scala @@ -17,13 +17,9 @@ package io.glutenproject.expression -import com.google.common.collect.Lists import io.glutenproject.substrait.expression.ExpressionNode import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types._ - -import scala.collection.mutable.ListBuffer class SubStringTransformer(str: Expression, pos: Expression, len: Expression, original: Expression) extends Substring(str: Expression, pos: Expression, len: Expression) diff --git a/jvm/src/main/scala/io/glutenproject/extension/ColumnarOverrides.scala b/jvm/src/main/scala/io/glutenproject/extension/ColumnarOverrides.scala index 2727d352d0d9..f7968e946ab7 100644 --- a/jvm/src/main/scala/io/glutenproject/extension/ColumnarOverrides.scala +++ b/jvm/src/main/scala/io/glutenproject/extension/ColumnarOverrides.scala @@ -18,9 +18,11 @@ package io.glutenproject.extension import io.glutenproject.{GlutenConfig, GlutenSparkExtensionsInjector} +import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.execution._ import io.glutenproject.extension.columnar.{RowGuard, TransformGuardRule} import org.apache.spark.SparkConf + import org.apache.spark.internal.Logging import org.apache.spark.sql.{SparkSession, SparkSessionExtensions} import org.apache.spark.sql.catalyst.expressions._ @@ -32,7 +34,6 @@ import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, V2CommandExec} import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, ArrowEvalPythonExecTransformer} import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.internal.SQLConf @@ -49,9 +50,9 @@ case class TransformPreOverrides() extends Rule[SparkPlan] { val actualPlan = plan.child logDebug(s"Columnar Processing for ${actualPlan.getClass} is under RowGuard.") actualPlan.withNewChildren(actualPlan.children.map(replaceWithTransformerPlan)) - case plan: ArrowEvalPythonExec => + /* case plan: ArrowEvalPythonExec => val columnarChild = replaceWithTransformerPlan(plan.child) - ArrowEvalPythonExecTransformer(plan.udfs, plan.resultAttrs, columnarChild, plan.evalType) + ArrowEvalPythonExecTransformer(plan.udfs, plan.resultAttrs, columnarChild, plan.evalType) */ case plan: BatchScanExec => logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") new BatchScanExecTransformer(plan.output, plan.scan) @@ -205,7 +206,7 @@ case class TransformPostOverrides() extends Rule[SparkPlan] { case plan: RowToColumnarExec => val child = replaceWithTransformerPlan(plan.child) logDebug(s"ColumnarPostOverrides RowToArrowColumnarExec(${child.getClass})") - RowToArrowColumnarExec(child) + BackendsApiManager.getSparkPlanExecApiInstance.genRowToArrowColumnarExec(child) case ColumnarToRowExec(child: ColumnarShuffleExchangeAdaptor) => replaceWithTransformerPlan(child) case ColumnarToRowExec(child: CoalesceBatchesExec) => @@ -214,8 +215,9 @@ case class TransformPostOverrides() extends Rule[SparkPlan] { if (columnarConf.enableNativeColumnarToRow) { val child = replaceWithTransformerPlan(plan.child) logDebug(s"ColumnarPostOverrides NativeColumnarToRowExec(${child.getClass})") - val nativeConversion = new NativeColumnarToRowExec(child) - if (nativeConversion.doValidate()) { + val nativeConversion = + BackendsApiManager.getSparkPlanExecApiInstance.genNativeColumnarToRowExec(child) + if (nativeConversion.doValidate()) { nativeConversion } else { logInfo("NativeColumnarToRow : Falling back to ColumnarToRow...") @@ -234,7 +236,8 @@ case class TransformPostOverrides() extends Rule[SparkPlan] { case c: ColumnarToRowExec => if (columnarConf.enableNativeColumnarToRow) { val child = replaceWithTransformerPlan(c.child) - val nativeConversion = new NativeColumnarToRowExec(child) + val nativeConversion = + BackendsApiManager.getSparkPlanExecApiInstance.genNativeColumnarToRowExec(child) if (nativeConversion.doValidate()) { nativeConversion } else { diff --git a/jvm/src/main/scala/io/glutenproject/extension/columnar/ColumnarGuardRule.scala b/jvm/src/main/scala/io/glutenproject/extension/columnar/ColumnarGuardRule.scala index 178f7be60e7c..f75a0bda9f0a 100644 --- a/jvm/src/main/scala/io/glutenproject/extension/columnar/ColumnarGuardRule.scala +++ b/jvm/src/main/scala/io/glutenproject/extension/columnar/ColumnarGuardRule.scala @@ -32,8 +32,6 @@ import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.python.ArrowEvalPythonExec -import org.apache.spark.sql.execution.python.ArrowEvalPythonExecTransformer import org.apache.spark.sql.execution.window.WindowExec // A guard to prevent a plan being converted into the plan transformer. @@ -75,11 +73,12 @@ case class TransformGuardRule() extends Rule[SparkPlan] { private def tryConvertToTransformer(plan: SparkPlan): Boolean = { try { plan match { - case plan: ArrowEvalPythonExec => + /* case plan: ArrowEvalPythonExec => if (!enableColumnarArrowUDF) return false val transformer = ArrowEvalPythonExecTransformer( plan.udfs, plan.resultAttrs, plan.child, plan.evalType) - transformer.doValidate() + if (!transformer.doValidate()) return false + transformer */ case plan: BatchScanExec => if (!enableColumnarBatchScan) return false val transformer = new BatchScanExecTransformer(plan.output, plan.scan) diff --git a/jvm/src/main/scala/org/apache/spark/shuffle/GlutenShuffleWriterWrapper.scala b/jvm/src/main/scala/org/apache/spark/shuffle/GlutenShuffleWriterWrapper.scala new file mode 100644 index 000000000000..afadee0aa502 --- /dev/null +++ b/jvm/src/main/scala/org/apache/spark/shuffle/GlutenShuffleWriterWrapper.scala @@ -0,0 +1,38 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle + +import io.glutenproject.backendsapi.BackendsApiManager + +import org.apache.spark.shuffle.sort.ColumnarShuffleHandle + +case class GlutenShuffleWriterWrapper[K, V](shuffleWriter: ShuffleWriter[K, V]) +case class GenShuffleWriterParameters[K, V](shuffleBlockResolver: IndexShuffleBlockResolver, + columnarShuffleHandle: ColumnarShuffleHandle[K, V], + mapId: Long, metrics: ShuffleWriteMetricsReporter) + +object GlutenShuffleWriterWrapper { + + def genColumnarShuffleWriter[K, V](shuffleBlockResolver: IndexShuffleBlockResolver, + columnarShuffleHandle: ColumnarShuffleHandle[K, V], + mapId: Long, metrics: ShuffleWriteMetricsReporter + ): ShuffleWriter[K, V] = + BackendsApiManager.getSparkPlanExecApiInstance + .genColumnarShuffleWriter(GenShuffleWriterParameters(shuffleBlockResolver, + columnarShuffleHandle, mapId, metrics)) + .shuffleWriter +} \ No newline at end of file diff --git a/jvm/src/main/scala/org/apache/spark/shuffle/sort/ColumnarShuffleManager.scala b/jvm/src/main/scala/org/apache/spark/shuffle/sort/ColumnarShuffleManager.scala index fba719f96d13..788ab6d14a07 100644 --- a/jvm/src/main/scala/org/apache/spark/shuffle/sort/ColumnarShuffleManager.scala +++ b/jvm/src/main/scala/org/apache/spark/shuffle/sort/ColumnarShuffleManager.scala @@ -20,7 +20,10 @@ package org.apache.spark.shuffle.sort import java.io.InputStream import java.util.concurrent.ConcurrentHashMap -import org.apache.spark._ +import scala.collection.JavaConverters._ + +import org.apache.spark.{ShuffleDependency, SparkConf, SparkEnv, TaskContext} + import org.apache.spark.internal.Logging import org.apache.spark.serializer.SerializerManager import org.apache.spark.shuffle._ @@ -29,8 +32,6 @@ import org.apache.spark.shuffle.sort.SortShuffleManager.canUseBatchFetch import org.apache.spark.storage.BlockId import org.apache.spark.util.collection.OpenHashSet -import scala.collection.JavaConverters._ - class ColumnarShuffleManager(conf: SparkConf) extends ShuffleManager with Logging { import ColumnarShuffleManager._ @@ -48,8 +49,8 @@ class ColumnarShuffleManager(conf: SparkConf) extends ShuffleManager with Loggin * Obtains a [[ShuffleHandle]] to pass to tasks. */ override def registerShuffle[K, V, C]( - shuffleId: Int, - dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { + shuffleId: Int, + dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { if (dependency.isInstanceOf[ColumnarShuffleDependency[_, _, _]]) { logInfo(s"Registering ColumnarShuffle shuffleId: ${shuffleId}") new ColumnarShuffleHandle[K, V]( @@ -77,17 +78,18 @@ class ColumnarShuffleManager(conf: SparkConf) extends ShuffleManager with Loggin /** Get a writer for a given partition. Called on executors by map tasks. */ override def getWriter[K, V]( - handle: ShuffleHandle, - mapId: Long, - context: TaskContext, - metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { + handle: ShuffleHandle, + mapId: Long, + context: TaskContext, + metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { val mapTaskIds = taskIdMapsForShuffle.computeIfAbsent(handle.shuffleId, _ => new OpenHashSet[Long](16)) mapTaskIds.synchronized { mapTaskIds.add(context.taskAttemptId()) } val env = SparkEnv.get handle match { case columnarShuffleHandle: ColumnarShuffleHandle[K @unchecked, V @unchecked] => - new ColumnarShuffleWriter(shuffleBlockResolver, columnarShuffleHandle, mapId, metrics) + GlutenShuffleWriterWrapper.genColumnarShuffleWriter( + shuffleBlockResolver, columnarShuffleHandle, mapId, metrics) case unsafeShuffleHandle: SerializedShuffleHandle[K @unchecked, V @unchecked] => new UnsafeShuffleWriter( env.blockManager, @@ -121,13 +123,13 @@ class ColumnarShuffleManager(conf: SparkConf) extends ShuffleManager with Loggin * Called on executors by reduce tasks. */ override def getReader[K, C]( - handle: ShuffleHandle, - startMapIndex: Int, - endMapIndex: Int, - startPartition: Int, - endPartition: Int, - context: TaskContext, - metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { val blocksByAddress = SparkEnv.get.mapOutputTracker .getMapSizesByExecutorId(handle.shuffleId, startMapIndex, endMapIndex, startPartition, endPartition) if (handle.isInstanceOf[ColumnarShuffleHandle[K, _]]) { @@ -189,6 +191,6 @@ object ColumnarShuffleManager extends Logging { } private[spark] class ColumnarShuffleHandle[K, V]( - shuffleId: Int, - dependency: ShuffleDependency[K, V, V]) - extends BaseShuffleHandle(shuffleId, dependency) {} + shuffleId: Int, + dependency: ShuffleDependency[K, V, V]) + extends BaseShuffleHandle(shuffleId, dependency) {} diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala b/jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala index c9dd2c374067..a3e5bd1d2076 100644 --- a/jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala +++ b/jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala @@ -20,33 +20,20 @@ package org.apache.spark.sql.execution import scala.collection.JavaConverters._ import scala.concurrent.Future -import io.glutenproject.GlutenConfig -import io.glutenproject.expression.{ConverterUtils, ExpressionConverter, ExpressionTransformer} -import io.glutenproject.substrait.expression.ExpressionNode -import io.glutenproject.substrait.rel.RelBuilder -import io.glutenproject.vectorized.{ArrowWritableColumnVector, ColumnarFactory, NativePartitioning} -import java.util -import org.apache.arrow.gandiva.expression.TreeNode -import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema} +import io.glutenproject.backendsapi.BackendsApiManager import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer -import org.apache.spark.shuffle.{ColumnarShuffleDependency, ShuffleHandle} +import org.apache.spark.shuffle.ShuffleHandle import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, UnsafeProjection} -import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.CoalesceExec.EmptyPartition -import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils import org.apache.spark.sql.execution.exchange._ -import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.createShuffleWriteProcessor import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.MutablePair case class ColumnarShuffleExchangeExec(override val outputPartitioning: Partitioning, child: SparkPlan, @@ -81,33 +68,13 @@ case class ColumnarShuffleExchangeExec(override val outputPartitioning: Partitio // super.stringArgs ++ Iterator(output.map(o => s"${o}#${o.dataType.simpleString}")) def doValidate(): Boolean = { - if (GlutenConfig.getConf.isClickHouseBackend) return true - // check input datatype - for (attr <- child.output) { - try { - ConverterUtils.createArrowField(attr) - } catch { - case e: UnsupportedOperationException => - logInfo(s"${attr.dataType} is not supported in ColumnarShuffledExchangeExec.") - return false - } - } - outputPartitioning match { - case HashPartitioning(exprs, n) => - if (!GlutenConfig.getConf.isClickHouseBackend) { - exprs.foreach(expr => { - if (!expr.isInstanceOf[Attribute]) { - logInfo("Expressions are not supported in HashPartitioning.") - return false - }}) - } - case _ => - } - true + BackendsApiManager.getTransformerApiInstance.validateColumnarShuffleExchangeExec( + outputPartitioning, child.output) } - val serializer: Serializer = ColumnarFactory.createColumnarBatchSerializer( - schema, + val serializer: Serializer = + BackendsApiManager.getSparkPlanExecApiInstance + .createColumnarBatchSerializer(schema, longMetric("avgReadBatchNumRows"), longMetric("numOutputRows")) @@ -215,8 +182,8 @@ class ColumnarShuffleExchangeAdaptor(override val outputPartitioning: Partitioni super.stringArgs ++ Iterator(s"[id=#$id]") //super.stringArgs ++ Iterator(output.map(o => s"${o}#${o.dataType.simpleString}")) - val serializer: Serializer = ColumnarFactory.createColumnarBatchSerializer( - schema, + val serializer: Serializer = BackendsApiManager.getSparkPlanExecApiInstance + .createColumnarBatchSerializer(schema, longMetric("avgReadBatchNumRows"), longMetric("numOutputRows")) @@ -328,196 +295,18 @@ object ColumnarShuffleExchangeExec extends Logging { spillTime: SQLMetric, compressTime: SQLMetric, prepareTime: SQLMetric): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = { - val arrowFields = outputAttributes.map(attr => ConverterUtils.createArrowField(attr)) - def serializeSchema(fields: Seq[Field]): Array[Byte] = { - val schema = new Schema(fields.asJava) - ConverterUtils.getSchemaBytesBuf(schema) - } - - // only used for fallback range partitioning - val rangePartitioner: Option[Partitioner] = newPartitioning match { - case RangePartitioning(sortingExpressions, numPartitions) => - // Extract only fields used for sorting to avoid collecting large fields that does not - // affect sorting result when deciding partition bounds in RangePartitioner - val rddForSampling = rdd.mapPartitionsInternal { iter => - // Internally, RangePartitioner runs a job on the RDD that samples keys to compute - // partition bounds. To get accurate samples, we need to copy the mutable keys. - iter.flatMap(batch => { - val rows = batch.rowIterator.asScala - val projection = - UnsafeProjection.create(sortingExpressions.map(_.child), outputAttributes) - val mutablePair = new MutablePair[InternalRow, Null]() - rows.map(row => mutablePair.update(projection(row).copy(), null)) - }) - } - // Construct ordering on extracted sort key. - val orderingAttributes = sortingExpressions.zipWithIndex.map { - case (ord, i) => - ord.copy(child = BoundReference(i, ord.dataType, ord.nullable)) - } - implicit val ordering = new LazilyGeneratedOrdering(orderingAttributes) - val part = new RangePartitioner( - numPartitions, - rddForSampling, - ascending = true, - samplePointsPerPartitionHint = SQLConf.get.rangeExchangeSampleSizePerPartition) - Some(part) - case _ => None - } - - // only used for fallback range partitioning - def computeAndAddPartitionId( - cbIter: Iterator[ColumnarBatch], - partitionKeyExtractor: InternalRow => Any): CloseablePairedColumnarBatchIterator = { - CloseablePairedColumnarBatchIterator { - cbIter - .filter(cb => cb.numRows != 0 && cb.numCols != 0) - .map { cb => - val startTime = System.nanoTime() - val pidVec = ArrowWritableColumnVector - .allocateColumns(cb.numRows, new StructType().add("pid", IntegerType)) - .head - (0 until cb.numRows).foreach { i => - val row = cb.getRow(i) - val pid = rangePartitioner.get.getPartition(partitionKeyExtractor(row)) - pidVec.putInt(i, pid) - } - - val newColumns = (pidVec +: (0 until cb.numCols).map(cb.column)).toArray - newColumns.foreach( - _.asInstanceOf[ArrowWritableColumnVector].getValueVector.setValueCount(cb.numRows)) - computePidTime.add(System.nanoTime() - startTime) - (0, new ColumnarBatch(newColumns, cb.numRows)) - } - } - } - - val nativePartitioning: NativePartitioning = newPartitioning match { - case SinglePartition => new NativePartitioning("single", 1, serializeSchema(arrowFields)) - case RoundRobinPartitioning(n) => - new NativePartitioning("rr", n, serializeSchema(arrowFields)) - case HashPartitioning(exprs, n) => - if (!GlutenConfig.getConf.isClickHouseBackend) { - // Function map is not expected to be used. - val functionMap = new java.util.HashMap[String, java.lang.Long]() - val exprNodeList = new util.ArrayList[ExpressionNode]() - exprs.foreach(expr => { - exprNodeList.add(ExpressionConverter - .replaceWithExpressionTransformer(expr, outputAttributes) - .asInstanceOf[ExpressionTransformer] - .doTransform(functionMap)) - }) - val projectRel = RelBuilder.makeProjectRel(null, exprNodeList) - new NativePartitioning( - "hash", - n, - serializeSchema(arrowFields), - projectRel.toProtobuf().toByteArray) - } else { - val fields = exprs.zipWithIndex.map { - case (expr, i) => - val treeNode: TreeNode = null - val attr = ConverterUtils.getAttrFromExpr(expr) - attr.name - } - new NativePartitioning( - "hash", - n, - null, - fields.mkString(",").getBytes("UTF-8")) - } - // range partitioning fall back to row-based partition id computation - case RangePartitioning(orders, n) => - val pidField = Field.nullable("pid", new ArrowType.Int(32, true)) - new NativePartitioning("range", n, serializeSchema(pidField +: arrowFields)) - } - - val isRoundRobin = newPartitioning.isInstanceOf[RoundRobinPartitioning] && - newPartitioning.numPartitions > 1 - - // RDD passed to ShuffleDependency should be the form of key-value pairs. - // ColumnarShuffleWriter will compute ids from ColumnarBatch on native side other than read the "key" part. - // Thus in Columnar Shuffle we never use the "key" part. - val isOrderSensitive = isRoundRobin && !SQLConf.get.sortBeforeRepartition - - val rddWithDummyKey: RDD[Product2[Int, ColumnarBatch]] = newPartitioning match { - case RangePartitioning(sortingExpressions, _) => - rdd.mapPartitionsWithIndexInternal((_, cbIter) => { - val partitionKeyExtractor: InternalRow => Any = { - val projection = - UnsafeProjection.create(sortingExpressions.map(_.child), outputAttributes) - row => projection(row) - } - val newIter = computeAndAddPartitionId(cbIter, partitionKeyExtractor) - - SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit] { _ => - newIter.closeAppendedVector() - } - - newIter - }, isOrderSensitive = isOrderSensitive) - case _ => - rdd.mapPartitionsWithIndexInternal( - (_, cbIter) => - cbIter.map { cb => - if (!GlutenConfig.getConf.isClickHouseBackend) { - (0 until cb.numCols).foreach( - cb.column(_) - .asInstanceOf[ArrowWritableColumnVector] - .getValueVector - .setValueCount(cb.numRows)) - } - (0, cb) - }, - isOrderSensitive = isOrderSensitive) - } - - val dependency = - new ColumnarShuffleDependency[Int, ColumnarBatch, ColumnarBatch]( - rddWithDummyKey, - new PartitionIdPassthrough(newPartitioning.numPartitions), - serializer, - shuffleWriterProcessor = createShuffleWriteProcessor(writeMetrics), - nativePartitioning = nativePartitioning, - dataSize = dataSize, - bytesSpilled = bytesSpilled, - numInputRows = numInputRows, - computePidTime = computePidTime, - splitTime = splitTime, - spillTime = spillTime, - compressTime = compressTime, - prepareTime = prepareTime) - - dependency - } -} - -case class CloseablePairedColumnarBatchIterator(iter: Iterator[(Int, ColumnarBatch)]) - extends Iterator[(Int, ColumnarBatch)] - with Logging { - - private var cur: (Int, ColumnarBatch) = _ - - def closeAppendedVector(): Unit = { - if (cur != null) { - logDebug("Close appended partition id vector") - cur match { - case (_, cb: ColumnarBatch) => - cb.column(0).asInstanceOf[ArrowWritableColumnVector].close() - } - cur = null - } - } - - override def hasNext: Boolean = { - iter.hasNext - } - - override def next(): (Int, ColumnarBatch) = { - closeAppendedVector() - if (iter.hasNext) { - cur = iter.next() - cur - } else Iterator.empty.next() + BackendsApiManager.getSparkPlanExecApiInstance.genShuffleDependency(rdd, + outputAttributes, + newPartitioning: Partitioning, + serializer: Serializer, + writeMetrics, + dataSize: SQLMetric, + bytesSpilled: SQLMetric, + numInputRows: SQLMetric, + computePidTime: SQLMetric, + splitTime: SQLMetric, + spillTime: SQLMetric, + compressTime: SQLMetric, + prepareTime: SQLMetric) } } diff --git a/jvm/src/test/java/io/glutenproject/backendsapi/TestBackendsApisSuite.java b/jvm/src/test/java/io/glutenproject/backendsapi/TestBackendsApisSuite.java new file mode 100644 index 000000000000..0ff7ec03d49e --- /dev/null +++ b/jvm/src/test/java/io/glutenproject/backendsapi/TestBackendsApisSuite.java @@ -0,0 +1,37 @@ +/* + * 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 io.glutenproject.backendsapi; + +import org.junit.Assert; +import org.junit.Test; + +public class TestBackendsApisSuite { + + @Test + public void testInitializeBackendsApi() { + BackendsApiManager.initialize("default"); + Assert.assertNotNull(BackendsApiManager.getIteratorApiInstance()); + Assert.assertNotNull(BackendsApiManager.getTransformerApiInstance()); + Assert.assertNotNull(BackendsApiManager.getSparkPlanExecApiInstance()); + } + + @Test(expected = AssertionError.class) + public void testNullBackendsApi() { + BackendsApiManager.initialize("none"); + } +} diff --git a/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.IIteratorApi b/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.IIteratorApi new file mode 100644 index 000000000000..5de95b463c31 --- /dev/null +++ b/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.IIteratorApi @@ -0,0 +1 @@ +io.glutenproject.backendsapi.IteratorApiImplSuite \ No newline at end of file diff --git a/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.ISparkPlanExecApi b/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.ISparkPlanExecApi new file mode 100644 index 000000000000..67062568046c --- /dev/null +++ b/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.ISparkPlanExecApi @@ -0,0 +1 @@ +io.glutenproject.backendsapi.SparkPlanExecApiImplSuite \ No newline at end of file diff --git a/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.ITransformerApi b/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.ITransformerApi new file mode 100644 index 000000000000..2eef799617c7 --- /dev/null +++ b/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.ITransformerApi @@ -0,0 +1 @@ +io.glutenproject.backendsapi.TransformerApiImplSuite \ No newline at end of file diff --git a/jvm/src/test/scala/io/glutenproject/backendsapi/IteratorApiImplSuite.scala b/jvm/src/test/scala/io/glutenproject/backendsapi/IteratorApiImplSuite.scala new file mode 100644 index 000000000000..10e5ef110671 --- /dev/null +++ b/jvm/src/test/scala/io/glutenproject/backendsapi/IteratorApiImplSuite.scala @@ -0,0 +1,114 @@ +/* + * 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 io.glutenproject.backendsapi + +import io.glutenproject.GlutenNumaBindingInfo +import io.glutenproject.execution.AbstractColumnarNativeIterator +import io.glutenproject.execution.BaseNativeFilePartition +import io.glutenproject.execution.WholestageTransformContext +import io.glutenproject.substrait.plan.PlanNode +import io.glutenproject.vectorized.AbstractBatchIterator +import io.glutenproject.vectorized.ExpressionEvaluator +import io.glutenproject.vectorized.ExpressionEvaluatorJniWrapper +import org.apache.spark.SparkConf +import org.apache.spark.TaskContext +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.vectorized.ColumnarBatch + +class IteratorApiImplSuite extends IIteratorApi { + /** + * Generate native row partition. + * + * @return + */ + override def genNativeFilePartition(p: InputPartition, + wsCxt: WholestageTransformContext + ): BaseNativeFilePartition = null + + /** + * Generate Iterator[ColumnarBatch] for CoalesceBatchesExec. + * + * @return + */ + override def genCoalesceIterator(iter: Iterator[ColumnarBatch], recordsPerBatch: Int, + numOutputRows: SQLMetric, numInputBatches: SQLMetric, + numOutputBatches: SQLMetric, collectTime: SQLMetric, + concatTime: SQLMetric, avgCoalescedNumRows: SQLMetric + ): Iterator[ColumnarBatch] = null + + /** + * Generate closeable ColumnBatch iterator. + * + * @return + */ + override def genCloseableColumnBatchIterator(iter: Iterator[ColumnarBatch] + ): Iterator[ColumnarBatch] = null + + /** + * Generate Iterator[ColumnarBatch] for first stage. + * + * @return + */ + override def genFirstStageIterator(inputPartition: BaseNativeFilePartition, + loadNative: Boolean, outputAttributes: Seq[Attribute], + context: TaskContext, jarList: Seq[String] + ): Iterator[ColumnarBatch] = null + + /** + * Generate Iterator[ColumnarBatch] for final stage. + * + * @return + */ + override def genFinalStageIterator(iter: Iterator[ColumnarBatch], + numaBindingInfo: GlutenNumaBindingInfo, + listJars: Seq[String], signature: String, + sparkConf: SparkConf, outputAttributes: Seq[Attribute], + rootNode: PlanNode, streamedSortPlan: SparkPlan, + pipelineTime: SQLMetric, buildRelationBatchHolder: Seq[ColumnarBatch], + dependentKernels: Seq[ExpressionEvaluator], + dependentKernelIterators: Seq[AbstractBatchIterator] + ): Iterator[ColumnarBatch] = null + + /** + * Generate columnar native iterator. + * + * @return + */ + override def genColumnarNativeIterator(delegated: Iterator[ColumnarBatch] + ): AbstractColumnarNativeIterator = null + + /** + * Generate BatchIterator for ExpressionEvaluator. + * + * @return + */ + override def genBatchIterator(wsPlan: Array[Byte], + iterList: Seq[AbstractColumnarNativeIterator], + jniWrapper: ExpressionEvaluatorJniWrapper + ): AbstractBatchIterator = null + + /** + * Get the backend api name. + * + * @return + */ + override def getBackendName: String = "default" +} diff --git a/jvm/src/test/scala/io/glutenproject/backendsapi/SparkPlanExecApiImplSuite.scala b/jvm/src/test/scala/io/glutenproject/backendsapi/SparkPlanExecApiImplSuite.scala new file mode 100644 index 000000000000..ee262c128e01 --- /dev/null +++ b/jvm/src/test/scala/io/glutenproject/backendsapi/SparkPlanExecApiImplSuite.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.backendsapi + +import io.glutenproject.execution.{NativeColumnarToRowExec, RowToArrowColumnarExec} +import org.apache.spark.ShuffleDependency + +import org.apache.spark.rdd.RDD +import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.{GenShuffleWriterParameters, GlutenShuffleWriterWrapper} +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch + +class SparkPlanExecApiImplSuite extends ISparkPlanExecApi { + /** + * Generate NativeColumnarToRowExec. + * + * @param child + * @return + */ + override def genNativeColumnarToRowExec(child: SparkPlan): NativeColumnarToRowExec = null + + /** + * Generate RowToArrowColumnarExec. + * + * @param child + * @return + */ + override def genRowToArrowColumnarExec(child: SparkPlan): RowToArrowColumnarExec = null + + /** + * Generate ShuffleDependency for ColumnarShuffleExchangeExec. + * + * @return + */ + override def genShuffleDependency(rdd: RDD[ColumnarBatch], outputAttributes: Seq[Attribute], + newPartitioning: Partitioning, serializer: Serializer, + writeMetrics: Map[String, SQLMetric], dataSize: SQLMetric, + bytesSpilled: SQLMetric, numInputRows: SQLMetric, + computePidTime: SQLMetric, splitTime: SQLMetric, + spillTime: SQLMetric, compressTime: SQLMetric, + prepareTime: SQLMetric + ): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = null + + /** + * Generate ColumnarShuffleWriter for ColumnarShuffleManager. + * + * @return + */ + override def genColumnarShuffleWriter[K, V](parameters: GenShuffleWriterParameters[K, V] + ): GlutenShuffleWriterWrapper[K, V] = null + + /** + * Generate ColumnarBatchSerializer for ColumnarShuffleExchangeExec. + * + * @return + */ + override def createColumnarBatchSerializer(schema: StructType, + readBatchNumRows: SQLMetric, + numOutputRows: SQLMetric): Serializer = null + + /** + * Get the backend api name. + * + * @return + */ + override def getBackendName: String = "default" +} diff --git a/jvm/src/test/scala/io/glutenproject/backendsapi/TransformerApiImplSuite.scala b/jvm/src/test/scala/io/glutenproject/backendsapi/TransformerApiImplSuite.scala new file mode 100644 index 000000000000..ddf01467aae1 --- /dev/null +++ b/jvm/src/test/scala/io/glutenproject/backendsapi/TransformerApiImplSuite.scala @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.glutenproject.backendsapi + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.Partitioning + +class TransformerApiImplSuite extends ITransformerApi { + + /** + * Do validate for ColumnarShuffleExchangeExec. + * + * @return + */ + override def validateColumnarShuffleExchangeExec(outputPartitioning: Partitioning, + outputAttributes: Seq[Attribute] + ): Boolean = false + + /** + * Get the backend api name. + * + * @return + */ + override def getBackendName: String = "default" +} diff --git a/jvm/src/test/scala/io/glutenproject/benchmarks/BenchmarkTest.scala b/jvm/src/test/scala/io/glutenproject/benchmarks/BenchmarkTest.scala index 2dc971feed52..1cb6feac6044 100644 --- a/jvm/src/test/scala/io/glutenproject/benchmarks/BenchmarkTest.scala +++ b/jvm/src/test/scala/io/glutenproject/benchmarks/BenchmarkTest.scala @@ -72,7 +72,7 @@ object BenchmarkTest { .config(GlutenConfig.GLUTEN_LOAD_NATIVE, "true") .config(GlutenConfig.GLUTEN_LOAD_ARROW, "false") .config(GlutenConfig.GLUTEN_LIB_PATH, - "/home/myubuntu/Works/c_cpp_projects/Kyligence-ClickHouse-MergeTree/cmake-build-release/utils/local-engine/liblocal_engine_jni.so") + "/usr/local/clickhouse/lib/libch.so") .config("spark.gluten.sql.columnar.iterator", "false") .config("spark.gluten.sql.columnar.ch.mergetree.enabled", "true") .config("spark.gluten.sql.columnar.ch.mergetree.table.path", diff --git a/jvm/src/test/scala/io/glutenproject/benchmarks/TableBenchmarkTest.scala b/jvm/src/test/scala/io/glutenproject/benchmarks/TableBenchmarkTest.scala index 9fe4fa487d9a..4aa594604b89 100644 --- a/jvm/src/test/scala/io/glutenproject/benchmarks/TableBenchmarkTest.scala +++ b/jvm/src/test/scala/io/glutenproject/benchmarks/TableBenchmarkTest.scala @@ -93,7 +93,7 @@ object TableBenchmarkTest { .config(GlutenConfig.GLUTEN_LOAD_NATIVE, "true") .config(GlutenConfig.GLUTEN_LOAD_ARROW, "false") .config(GlutenConfig.GLUTEN_LIB_PATH, - "/home/myubuntu/Works/c_cpp_projects/Kyligence-ClickHouse-MergeTree/cmake-build-release/utils/local-engine/liblocal_engine_jni.so") + "/usr/local/clickhouse/lib/libch.so") .config("spark.gluten.sql.columnar.iterator", "false") .config("spark.gluten.sql.columnar.ch.mergetree.enabled", "true") .config("spark.gluten.sql.columnar.ch.mergetree.table.path", diff --git a/pom.xml b/pom.xml index 7b234d51e891..e489020a04d7 100644 --- a/pom.xml +++ b/pom.xml @@ -34,6 +34,34 @@ shims + + 2.12.10 + 2.12 + 3.1.1 + 8.0.0-gluten-SNAPSHOT + arrow-memory-unsafe + ${hadoop.version} + UTF-8 + UTF-8 + ${project.basedir}/../tools + ${project.basedir}/../tools + OFF + OFF + OFF + OFF + /usr/local + ON + ON + OFF + OFF + OFF + ${project.basedir}/../tools/build_velox/velox_ep + spark-sql-columnar + Gluten + gluten + 3.1.1 + + spark @@ -107,34 +135,35 @@ arrow-memory-netty - - - 2.12.10 - 2.12 - 3.1.1 - 8.0.0-gluten-SNAPSHOT - arrow-memory-unsafe - ${hadoop.version} - UTF-8 - UTF-8 - ${project.basedir}/../tools - ${project.basedir}/../tools - OFF - OFF - OFF - OFF - /usr/local - ON - ON - OFF - OFF - OFF - ${project.basedir}/../tools/build_velox/velox_ep - spark-sql-columnar - Gluten - 3.1.1 - + + backends-clickhouse + + false + + + backends-clickhouse + + + + backends-velox + + false + + + backends-velox + + + + backends-gazelle + + false + + + backends-velox + + +