Skip to content
This repository has been archived by the owner on Sep 18, 2023. It is now read-only.

Commit

Permalink
[NSE-631] Do not write schema in shuffle writting (#632)
Browse files Browse the repository at this point in the history
* [NSE-631] Do not write schema in shuffle writting

* fix

* fix

* fix

* fix

* debugging

Closes #631
  • Loading branch information
zhztheplayer authored Dec 16, 2021
1 parent 2dd1c52 commit f0655cc
Show file tree
Hide file tree
Showing 11 changed files with 136 additions and 36 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -41,17 +41,34 @@
* This class reads from an input stream containing compressed buffers and produces
* ArrowRecordBatches.
*/
public class ArrowCompressedStreamReader extends ArrowStreamReader {
public class SchemaAwareArrowCompressedStreamReader extends ArrowStreamReader {
private final Schema originalSchema;
private String compressType;

public ArrowCompressedStreamReader(InputStream in, BufferAllocator allocator) {
public SchemaAwareArrowCompressedStreamReader(Schema originalSchema, InputStream in,
BufferAllocator allocator) {
super(in, allocator);
this.originalSchema = originalSchema;
}


public SchemaAwareArrowCompressedStreamReader(InputStream in,
BufferAllocator allocator) {
this(null, in, allocator);
}

public String GetCompressType() {
return compressType;
}

@Override
protected Schema readSchema() throws IOException {
if (originalSchema == null) {
return super.readSchema();
}
return originalSchema;
}

protected void initialize() throws IOException {
Schema originalSchema = readSchema();
List<Field> fields = new ArrayList<>();
Expand Down
Original file line number Diff line number Diff line change
@@ -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 com.intel.oap.vectorized;

import org.apache.arrow.memory.BufferAllocator;
import org.apache.arrow.vector.ipc.ArrowStreamReader;
import org.apache.arrow.vector.types.pojo.Schema;

import java.io.IOException;
import java.io.InputStream;

public class SchemaAwareArrowStreamReader extends ArrowStreamReader {
private final Schema originalSchema;

public SchemaAwareArrowStreamReader(Schema originalSchema, InputStream in,
BufferAllocator allocator) {
super(in, allocator);
this.originalSchema = originalSchema;
}

public SchemaAwareArrowStreamReader(InputStream in,
BufferAllocator allocator) {
this(null, in, allocator);
}

@Override
protected Schema readSchema() throws IOException {
if (originalSchema == null) {
return super.readSchema();
}
return originalSchema;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,8 @@ public long make(
int subDirsPerLocalDir,
String localDirs,
boolean preferSpill,
long memoryPoolId) {
long memoryPoolId,
boolean writeSchema) {
return nativeMake(
part.getShortName(),
part.getNumPartitions(),
Expand All @@ -60,7 +61,8 @@ public long make(
subDirsPerLocalDir,
localDirs,
preferSpill,
memoryPoolId);
memoryPoolId,
writeSchema);
}

public native long nativeMake(
Expand All @@ -75,7 +77,8 @@ public native long nativeMake(
int subDirsPerLocalDir,
String localDirs,
boolean preferSpill,
long memoryPoolId);
long memoryPoolId,
boolean writeSchema);

/**
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -180,6 +180,9 @@ class GazellePluginConfig(conf: SQLConf) extends Logging {
val columnarShufflePreferSpill: Boolean =
conf.getConfString("spark.oap.sql.columnar.shuffle.preferSpill", "true").toBoolean

val columnarShuffleWriteSchema: Boolean =
conf.getConfString("spark.oap.sql.columnar.shuffle.writeSchema", "false").toBoolean

// The supported customized compression codec is lz4 and fastpfor.
val columnarShuffleUseCustomizedCompressionCodec: String =
conf.getConfString("spark.oap.sql.columnar.shuffle.customizedCompression.codec", "lz4")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,13 +25,15 @@ import scala.collection.mutable
import scala.collection.mutable.ListBuffer
import scala.reflect.ClassTag

import com.intel.oap.GazellePluginConfig
import com.intel.oap.expression.ConverterUtils
import org.apache.arrow.dataset.jni.UnsafeRecordBatchSerializer
import org.apache.arrow.memory.ArrowBuf
import org.apache.arrow.memory.BufferAllocator
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
Expand All @@ -42,19 +44,25 @@ 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.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

class ArrowColumnarBatchSerializer(readBatchNumRows: SQLMetric, numOutputRows: SQLMetric)
extends Serializer
with Serializable {
class ArrowColumnarBatchSerializer(
schema: StructType, readBatchNumRows: SQLMetric, numOutputRows: SQLMetric)
extends Serializer with Serializable {

/** Creates a new [[SerializerInstance]]. */
override def newInstance(): SerializerInstance =
new ArrowColumnarBatchSerializerInstance(readBatchNumRows, numOutputRows)
override def newInstance(): SerializerInstance = {
val arrowSchema = ArrowUtils.toArrowSchema(schema, SQLConf.get.sessionLocalTimeZone)
new ArrowColumnarBatchSerializerInstance(arrowSchema, readBatchNumRows, numOutputRows)
}
}

private class ArrowColumnarBatchSerializerInstance(
schema: Schema,
readBatchNumRows: SQLMetric,
numOutputRows: SQLMetric)
extends SerializerInstance
Expand All @@ -63,6 +71,8 @@ private class ArrowColumnarBatchSerializerInstance(
override def deserializeStream(in: InputStream): DeserializationStream = {
new DeserializationStream {

private val readSchema = GazellePluginConfig.getConf.columnarShuffleWriteSchema

private val compressionEnabled =
SparkEnv.get.conf.getBoolean("spark.shuffle.compress", true)

Expand Down Expand Up @@ -148,10 +158,11 @@ private class ArrowColumnarBatchSerializerInstance(
throw new EOFException
}
} else {
val suggestedSchema = if (readSchema) null else schema
if (compressionEnabled) {
reader = new ArrowCompressedStreamReader(in, allocator)
reader = new SchemaAwareArrowCompressedStreamReader(suggestedSchema, in, allocator)
} else {
reader = new ArrowStreamReader(in, allocator)
reader = new SchemaAwareArrowStreamReader(suggestedSchema, in, allocator)
}
try {
root = reader.getVectorSchemaRoot
Expand Down Expand Up @@ -220,7 +231,7 @@ private class ArrowColumnarBatchSerializerInstance(

val serializedBatch = jniWrapper.decompress(
schemaHolderId,
reader.asInstanceOf[ArrowCompressedStreamReader].GetCompressType(),
reader.asInstanceOf[SchemaAwareArrowCompressedStreamReader].GetCompressType(),
root.getRowCount,
bufAddrs.toArray,
bufSizes.toArray,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,8 @@ class ColumnarShuffleWriter[K, V](
}
private val preferSpill = GazellePluginConfig.getConf.columnarShufflePreferSpill

private val writeSchema = GazellePluginConfig.getConf.columnarShuffleWriteSchema

private val jniWrapper = new ShuffleSplitterJniWrapper()

private var nativeSplitter: Long = 0
Expand Down Expand Up @@ -116,7 +118,8 @@ class ColumnarShuffleWriter[K, V](
// fixme pass true when being called by self
return jniWrapper.nativeSpill(nativeSplitter, size, false)
}
}).getNativeInstanceId)
}).getNativeInstanceId,
writeSchema)
}

while (records.hasNext) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,19 +18,11 @@
package org.apache.spark.sql.execution

import com.google.common.collect.Lists
import com.intel.oap.expression.{
CodeGeneration,
ColumnarExpression,
ColumnarExpressionConverter,
ConverterUtils
}
import com.intel.oap.vectorized.{
ArrowColumnarBatchSerializer,
ArrowWritableColumnVector,
NativePartitioning
}
import com.intel.oap.expression.{CodeGeneration, ColumnarExpression, ColumnarExpressionConverter, ConverterUtils}
import com.intel.oap.vectorized.{ArrowColumnarBatchSerializer, ArrowWritableColumnVector, NativePartitioning}
import org.apache.arrow.gandiva.expression.TreeBuilder
import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema}

import org.apache.spark._
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
Expand All @@ -46,20 +38,16 @@ import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils
import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.createShuffleWriteProcessor
import org.apache.spark.sql.execution.exchange._
import org.apache.spark.sql.execution.metric.{
SQLMetric,
SQLMetrics,
SQLShuffleReadMetricsReporter,
SQLShuffleWriteMetricsReporter
}
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, Utils}

import scala.collection.JavaConverters._
import scala.concurrent.Future

import org.apache.spark.sql.util.ArrowUtils

case class ColumnarShuffleExchangeExec(
override val outputPartitioning: Partitioning,
child: SparkPlan,
Expand Down Expand Up @@ -108,6 +96,7 @@ case class ColumnarShuffleExchangeExec(
}

val serializer: Serializer = new ArrowColumnarBatchSerializer(
schema,
longMetric("avgReadBatchNumRows"),
longMetric("numOutputRows"))

Expand Down Expand Up @@ -206,6 +195,7 @@ class ColumnarShuffleExchangeAdaptor(
//super.stringArgs ++ Iterator(output.map(o => s"${o}#${o.dataType.simpleString}"))

val serializer: Serializer = new ArrowColumnarBatchSerializer(
schema,
longMetric("avgReadBatchNumRows"),
longMetric("numOutputRows"))

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,19 @@
package com.intel.oap.vectorized

import java.io.FileInputStream
import java.util

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.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.BooleanType
import org.apache.spark.sql.types.IntegerType
import org.apache.spark.sql.types.StringType
import org.apache.spark.sql.types.StructField
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.vectorized.ColumnarBatch
import org.apache.spark.{SparkConf, SparkFunSuite}

Expand All @@ -31,6 +41,7 @@ class ArrowColumnarBatchSerializerSuite extends SparkFunSuite with SharedSparkSe
override def sparkConf: SparkConf =
super.sparkConf
.set("spark.shuffle.compress", "false")
.set("spark.oap.sql.columnar.shuffle.writeSchema", "true")

override def beforeEach() = {
avgBatchNumRows = SQLMetrics.createAverageMetric(
Expand All @@ -43,7 +54,12 @@ class ArrowColumnarBatchSerializerSuite extends SparkFunSuite with SharedSparkSe
test("deserialize all null") {
val input = getTestResourcePath("test-data/native-splitter-output-all-null")
val serializer =
new ArrowColumnarBatchSerializer(avgBatchNumRows, outputNumRows).newInstance()
new ArrowColumnarBatchSerializer(
new StructType(
Array(StructField("f1", BooleanType), StructField("f2", IntegerType),
StructField("f3", StringType))),
avgBatchNumRows,
outputNumRows).newInstance()
val deserializedStream =
serializer.deserializeStream(new FileInputStream(input))

Expand Down Expand Up @@ -71,7 +87,11 @@ class ArrowColumnarBatchSerializerSuite extends SparkFunSuite with SharedSparkSe
test("deserialize nullable string") {
val input = getTestResourcePath("test-data/native-splitter-output-nullable-string")
val serializer =
new ArrowColumnarBatchSerializer(avgBatchNumRows, outputNumRows).newInstance()
new ArrowColumnarBatchSerializer(
new StructType(
Array(StructField("f1", BooleanType), StructField("f2", StringType),
StructField("f3", StringType))), avgBatchNumRows,
outputNumRows).newInstance()
val deserializedStream =
serializer.deserializeStream(new FileInputStream(input))

Expand Down
4 changes: 3 additions & 1 deletion native-sql-engine/cpp/src/jni/jni_wrapper.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1027,7 +1027,8 @@ Java_com_intel_oap_vectorized_ShuffleSplitterJniWrapper_nativeMake(
JNIEnv* env, jobject, jstring partitioning_name_jstr, jint num_partitions,
jbyteArray schema_arr, jbyteArray expr_arr, jlong offheap_per_task, jint buffer_size,
jstring compression_type_jstr, jstring data_file_jstr, jint num_sub_dirs,
jstring local_dirs_jstr, jboolean prefer_spill, jlong memory_pool_id) {
jstring local_dirs_jstr, jboolean prefer_spill, jlong memory_pool_id,
jboolean write_schema) {
JNI_METHOD_START
if (partitioning_name_jstr == NULL) {
JniThrow(std::string("Short partitioning name can't be null"));
Expand All @@ -1048,6 +1049,7 @@ Java_com_intel_oap_vectorized_ShuffleSplitterJniWrapper_nativeMake(
env->ReleaseStringUTFChars(partitioning_name_jstr, partitioning_name_c);

auto splitOptions = SplitOptions::Defaults();
splitOptions.write_schema = write_schema;
splitOptions.prefer_spill = prefer_spill;
if (buffer_size > 0) {
splitOptions.buffer_size = buffer_size;
Expand Down
4 changes: 3 additions & 1 deletion native-sql-engine/cpp/src/shuffle/splitter.cc
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,9 @@ class Splitter::PartitionWriter {
const auto& data_file_os = splitter_->data_file_os_;
ARROW_ASSIGN_OR_RAISE(auto before_write, data_file_os->Tell());

RETURN_NOT_OK(WriteSchemaPayload(data_file_os.get()));
if (splitter_->options_.write_schema) {
RETURN_NOT_OK(WriteSchemaPayload(data_file_os.get()));
}

if (spilled_file_opened_) {
RETURN_NOT_OK(spilled_file_os_->Close());
Expand Down
1 change: 1 addition & 0 deletions native-sql-engine/cpp/src/shuffle/type.h
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ struct SplitOptions {
int32_t num_sub_dirs = kDefaultNumSubDirs;
arrow::Compression::type compression_type = arrow::Compression::UNCOMPRESSED;
bool prefer_spill = true;
bool write_schema = true;

std::string data_file;

Expand Down

0 comments on commit f0655cc

Please sign in to comment.