This repository has been archived by the owner on Sep 18, 2023. It is now read-only.
-
Notifications
You must be signed in to change notification settings - Fork 75
[NSE-1075] Dynamically adjust input partition size #1076
Open
PHILO-HE
wants to merge
8
commits into
oap-project:main
Choose a base branch
from
PHILO-HE:partition-optimize
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
8 commits
Select commit
Hold shift + click to select a range
58996d6
Initial commit
PHILO-HE c5f51e9
Refine the code
PHILO-HE 758246f
Remove some changes
PHILO-HE 633a548
Make required config accessible
PHILO-HE de19cbc
Adjust partition size empirically
PHILO-HE 6fbb54a
Change upper/lower bound
PHILO-HE 66269c2
Port the code for getFilePartitions
PHILO-HE 0bc4f43
Make WIP code commented
PHILO-HE File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
29 changes: 29 additions & 0 deletions
29
...common/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/ScanUtils.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,29 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.sql.execution.datasources.v2.arrow | ||
|
||
import org.apache.spark.sql.catalyst.expressions.AttributeReference | ||
import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex | ||
|
||
|
||
object ScanUtils { | ||
|
||
def toAttributes(fileIndex: PartitioningAwareFileIndex): Seq[AttributeReference] = { | ||
fileIndex.partitionSchema.toAttributes | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -16,20 +16,27 @@ | |
*/ | ||
package com.intel.oap.spark.sql.execution.datasources.v2.arrow | ||
|
||
import scala.collection.JavaConverters._ | ||
import com.intel.oap.sql.shims.SparkShimLoader | ||
import java.util.Locale | ||
|
||
import scala.collection.JavaConverters._ | ||
import org.apache.hadoop.fs.Path | ||
|
||
import org.apache.spark.sql.SparkSession | ||
import org.apache.spark.sql.{AnalysisException, SparkSession} | ||
import org.apache.spark.sql.catalyst.expressions.Expression | ||
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection | ||
import org.apache.spark.sql.connector.read.PartitionReaderFactory | ||
import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex | ||
import org.apache.spark.sql.execution.PartitionedFileUtil | ||
import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionDirectory, PartitionedFile, PartitioningAwareFileIndex} | ||
import org.apache.spark.sql.execution.datasources.v2.FileScan | ||
import org.apache.spark.sql.execution.datasources.v2.arrow.ScanUtils | ||
import org.apache.spark.sql.internal.SQLConf | ||
import org.apache.spark.sql.sources.Filter | ||
import org.apache.spark.sql.types.StructType | ||
import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
import org.apache.spark.util.SerializableConfiguration | ||
|
||
import scala.collection.mutable.ArrayBuffer | ||
|
||
case class ArrowScan( | ||
sparkSession: SparkSession, | ||
fileIndex: PartitioningAwareFileIndex, | ||
|
@@ -41,6 +48,10 @@ case class ArrowScan( | |
dataFilters: Seq[Expression] = Seq.empty) | ||
extends FileScan { | ||
|
||
// Use the default value for org.apache.spark.internal.config.IO_WARNING_LARGEFILETHRESHOLD. | ||
val IO_WARNING_LARGEFILETHRESHOLD: Long = 1024 * 1024 * 1024 | ||
var openCostInBytesFinal = sparkSession.sessionState.conf.filesOpenCostInBytes | ||
|
||
override def isSplitable(path: Path): Boolean = { | ||
ArrowUtils.isOriginalFormatSplitable( | ||
new ArrowOptions(new CaseInsensitiveStringMap(options).asScala.toMap)) | ||
|
@@ -63,4 +74,146 @@ case class ArrowScan( | |
override def withFilters(partitionFilters: Seq[Expression], | ||
dataFilters: Seq[Expression]): FileScan = | ||
this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) | ||
|
||
// compute maxSplitBytes | ||
// def maxSplitBytes(sparkSession: SparkSession, | ||
// selectedPartitions: Seq[PartitionDirectory]): Long = { | ||
// // TODO: unify it with PREFERRED_PARTITION_SIZE_UPPER_BOUND. | ||
// val defaultMaxSplitBytes = sparkSession.sessionState.conf.filesMaxPartitionBytes | ||
// val openCostInBytes = sparkSession.sessionState.conf.filesOpenCostInBytes | ||
// // val minPartitionNum = sparkSession.sessionState.conf.filesMinPartitionNum | ||
// // .getOrElse(sparkSession.leafNodeDefaultParallelism) | ||
// val minPartitionNum = sparkSession.sessionState.conf.filesMinPartitionNum | ||
// .getOrElse(SparkShimLoader.getSparkShims.leafNodeDefaultParallelism(sparkSession)) | ||
// val PREFERRED_PARTITION_SIZE_LOWER_BOUND: Long = 256 * 1024 * 1024 | ||
// val PREFERRED_PARTITION_SIZE_UPPER_BOUND: Long = 1024 * 1024 * 1024 | ||
// val totalBytes = selectedPartitions.flatMap(_.files.map(_.getLen + openCostInBytes)).sum | ||
// var maxBytesPerCore = totalBytes / minPartitionNum | ||
// var bytesPerCoreFinal = maxBytesPerCore | ||
// var bytesPerCore = maxBytesPerCore | ||
// | ||
// if (bytesPerCore > PREFERRED_PARTITION_SIZE_UPPER_BOUND) { | ||
// // Adjust partition size. | ||
// var i = 2 | ||
// while (bytesPerCore > PREFERRED_PARTITION_SIZE_UPPER_BOUND) { | ||
// bytesPerCore = maxBytesPerCore / i | ||
// if (bytesPerCore > PREFERRED_PARTITION_SIZE_LOWER_BOUND) { | ||
// bytesPerCoreFinal = bytesPerCore | ||
// } | ||
// i = i + 1 | ||
// } | ||
// Math.min(PREFERRED_PARTITION_SIZE_UPPER_BOUND, bytesPerCoreFinal) | ||
// // Math.min(defaultMaxSplitBytes, Math.max(openCostInBytes, bytesPerCore)) | ||
// } else { | ||
// // adjust open cost. | ||
// var i = 2 | ||
// while (bytesPerCore < PREFERRED_PARTITION_SIZE_LOWER_BOUND) { | ||
// val dynamicOpenCostInBytes = openCostInBytesFinal * i | ||
// val totalBytes = | ||
// selectedPartitions.flatMap(_.files.map(_.getLen + dynamicOpenCostInBytes)).sum | ||
// maxBytesPerCore = totalBytes / minPartitionNum | ||
// if (maxBytesPerCore < PREFERRED_PARTITION_SIZE_UPPER_BOUND) { | ||
// openCostInBytesFinal = dynamicOpenCostInBytes | ||
// bytesPerCoreFinal = maxBytesPerCore | ||
// } | ||
// i = i + 1 | ||
// } | ||
// Math.max(PREFERRED_PARTITION_SIZE_LOWER_BOUND, bytesPerCoreFinal) | ||
// } | ||
// } | ||
|
||
// This implementation is ported from spark FilePartition.scala with changes for | ||
// adjusting openCost. | ||
def getFilePartitions(sparkSession: SparkSession, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @jackylee-ch, please put your code changes for open cost here. It should be workable. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Okey |
||
partitionedFiles: Seq[PartitionedFile], | ||
maxSplitBytes: Long): Seq[FilePartition] = { | ||
val partitions = new ArrayBuffer[FilePartition] | ||
val currentFiles = new ArrayBuffer[PartitionedFile] | ||
var currentSize = 0L | ||
|
||
/** Close the current partition and move to the next. */ | ||
def closePartition(): Unit = { | ||
if (currentFiles.nonEmpty) { | ||
// Copy to a new Array. | ||
val newPartition = FilePartition(partitions.size, currentFiles.toArray) | ||
partitions += newPartition | ||
} | ||
currentFiles.clear() | ||
currentSize = 0 | ||
} | ||
|
||
val openCostInBytes = sparkSession.sessionState.conf.filesOpenCostInBytes | ||
// Assign files to partitions using "Next Fit Decreasing" | ||
partitionedFiles.foreach { file => | ||
if (currentSize + file.length > maxSplitBytes) { | ||
closePartition() | ||
} | ||
// Add the given file to the current partition. | ||
currentSize += file.length + openCostInBytes | ||
currentFiles += file | ||
} | ||
closePartition() | ||
partitions.toSeq | ||
} | ||
|
||
// This implementation is ported from spark FileScan with only changes for computing | ||
// maxSplitBytes. | ||
// override def partitions: Seq[FilePartition] = { | ||
// val selectedPartitions = fileIndex.listFiles(partitionFilters, dataFilters) | ||
// // val maxSplitBytes = FilePartition.maxSplitBytes(sparkSession, selectedPartitions) | ||
// val maxSplitBytes = this.maxSplitBytes(sparkSession, selectedPartitions) | ||
// // val partitionAttributes = fileIndex.partitionSchema.toAttributes | ||
// val partitionAttributes = ScanUtils.toAttributes(fileIndex) | ||
// val attributeMap = partitionAttributes.map(a => normalizeName(a.name) -> a).toMap | ||
// val readPartitionAttributes = readPartitionSchema.map { readField => | ||
// attributeMap.get(normalizeName(readField.name)).getOrElse { | ||
// // throw QueryCompilationErrors.cannotFindPartitionColumnInPartitionSchemaError( | ||
// // readField, fileIndex.partitionSchema) | ||
// throw new RuntimeException(s"Can't find required partition column ${readField.name} " + | ||
// s"in partition schema ${fileIndex.partitionSchema}") | ||
// } | ||
// } | ||
// lazy val partitionValueProject = | ||
// GenerateUnsafeProjection.generate(readPartitionAttributes, partitionAttributes) | ||
// val splitFiles = selectedPartitions.flatMap { partition => | ||
// // Prune partition values if part of the partition columns are not required. | ||
// val partitionValues = if (readPartitionAttributes != partitionAttributes) { | ||
// partitionValueProject(partition.values).copy() | ||
// } else { | ||
// partition.values | ||
// } | ||
// partition.files.flatMap { file => | ||
// val filePath = file.getPath | ||
// PartitionedFileUtil.splitFiles( | ||
// sparkSession = sparkSession, | ||
// file = file, | ||
// filePath = filePath, | ||
// isSplitable = isSplitable(filePath), | ||
// maxSplitBytes = maxSplitBytes, | ||
// partitionValues = partitionValues | ||
// ) | ||
// }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) | ||
// } | ||
// | ||
// if (splitFiles.length == 1) { | ||
// val path = new Path(splitFiles(0).filePath) | ||
// if (!isSplitable(path) && splitFiles(0).length > | ||
// IO_WARNING_LARGEFILETHRESHOLD) { | ||
// logWarning(s"Loading one large unsplittable file ${path.toString} with only one " + | ||
// s"partition, the reason is: ${getFileUnSplittableReason(path)}") | ||
// } | ||
// } | ||
// | ||
// FilePartition.getFilePartitions(sparkSession, splitFiles, maxSplitBytes) | ||
// } | ||
|
||
private val isCaseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis | ||
|
||
private def normalizeName(name: String): String = { | ||
if (isCaseSensitive) { | ||
name | ||
} else { | ||
name.toLowerCase(Locale.ROOT) | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
28 changes: 28 additions & 0 deletions
28
shims/spark321/src/main/scala/org/apache/spark/sql/util/ShimUtils.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,28 @@ | ||
/* | ||
* Copyright 2020 Intel Corporation | ||
* | ||
* 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.util | ||
|
||
import org.apache.spark.sql.SparkSession | ||
import org.apache.spark.sql.internal.SQLConf | ||
|
||
object ShimUtils { | ||
|
||
def leafNodeDefaultParallelism(sparkSession: SparkSession): Int = { | ||
sparkSession.conf.get(SQLConf.LEAF_NODE_DEFAULT_PARALLELISM).getOrElse( | ||
sparkSession.sparkContext.defaultParallelism) | ||
} | ||
} |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@jackylee-ch, I note you have introduced a sort of computation for taskParallelismNum, is it as same as minPartitionNum? This piece of code is ported from spark source code.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No, they are not same. The
taskParallelismNum
is actually thespark.sql.files.expectedPartitionNum
, which can be configured by the user and the default value is the maximum number of tasks that can be parallelized in the current application.