Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -16,29 +16,22 @@
package io.delta.kernel.spark.read;

import io.delta.kernel.expressions.Predicate;
import io.delta.kernel.spark.utils.PartitionUtils;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import org.apache.hadoop.conf.Configuration;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.connector.read.Batch;
import org.apache.spark.sql.connector.read.InputPartition;
import org.apache.spark.sql.connector.read.PartitionReaderFactory;
import org.apache.spark.sql.execution.datasources.FileFormat$;
import org.apache.spark.sql.execution.datasources.FilePartition;
import org.apache.spark.sql.execution.datasources.FilePartition$;
import org.apache.spark.sql.execution.datasources.PartitionedFile;
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat;
import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils;
import org.apache.spark.sql.internal.SQLConf;
import org.apache.spark.sql.sources.Filter;
import org.apache.spark.sql.types.StructType;
import scala.Function1;
import scala.Option;
import scala.Tuple2;
import scala.collection.Iterator;
import scala.collection.JavaConverters;

public class SparkBatch implements Batch {
Expand Down Expand Up @@ -88,7 +81,9 @@ public SparkBatch(
@Override
public InputPartition[] planInputPartitions() {
SparkSession sparkSession = SparkSession.active();
long maxSplitBytes = calculateMaxSplitBytes(sparkSession);
long maxSplitBytes =
PartitionUtils.calculateMaxSplitBytes(
sparkSession, totalBytes, partitionedFiles.size(), sqlConf);

scala.collection.Seq<FilePartition> filePartitions =
FilePartition$.MODULE$.getFilePartitions(
Expand All @@ -98,25 +93,14 @@ public InputPartition[] planInputPartitions() {

@Override
public PartitionReaderFactory createReaderFactory() {
boolean enableVectorizedReader =
ParquetUtils.isBatchReadSupportedForSchema(sqlConf, readDataSchema);
scala.collection.immutable.Map<String, String> optionsWithBatch =
scalaOptions.$plus(
new Tuple2<>(
FileFormat$.MODULE$.OPTION_RETURNING_BATCH(),
String.valueOf(enableVectorizedReader)));
Function1<PartitionedFile, Iterator<InternalRow>> readFunc =
new ParquetFileFormat()
.buildReaderWithPartitionValues(
SparkSession.active(),
dataSchema,
partitionSchema,
readDataSchema,
JavaConverters.asScalaBuffer(Arrays.asList(dataFilters)).toSeq(),
optionsWithBatch,
hadoopConf);

return new SparkReaderFactory(readFunc, enableVectorizedReader);
return PartitionUtils.createParquetReaderFactory(
dataSchema,
partitionSchema,
readDataSchema,
dataFilters,
scalaOptions,
hadoopConf,
sqlConf);
}

@Override
Expand Down Expand Up @@ -145,24 +129,4 @@ public int hashCode() {
result = 31 * result + Integer.hashCode(partitionedFiles.size());
return result;
}

private long calculateMaxSplitBytes(SparkSession sparkSession) {
long defaultMaxSplitBytes = sqlConf.filesMaxPartitionBytes();
long openCostInBytes = sqlConf.filesOpenCostInBytes();
Option<Object> minPartitionNumOption = sqlConf.filesMinPartitionNum();

int minPartitionNum =
minPartitionNumOption.isDefined()
? ((Number) minPartitionNumOption.get()).intValue()
: sqlConf
.getConf(SQLConf.LEAF_NODE_DEFAULT_PARALLELISM())
.getOrElse(() -> sparkSession.sparkContext().defaultParallelism());
if (minPartitionNum <= 0) {
minPartitionNum = 1;
}
long calculatedTotalBytes = totalBytes + (long) partitionedFiles.size() * openCostInBytes;
long bytesPerCore = calculatedTotalBytes / minPartitionNum;

return Math.min(defaultMaxSplitBytes, Math.max(openCostInBytes, bytesPerCore));
}
}
Loading