Skip to content
Merged
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 @@ -27,7 +27,6 @@ import org.apache.spark.sql.delta.hooks.AutoCompact
import org.apache.spark.sql.delta.perf.{DeltaOptimizedWriterExec, GlutenDeltaOptimizedWriterExec}
import org.apache.spark.sql.delta.schema.InnerInvariantViolationException
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.stats.{GlutenDeltaIdentityColumnStatsTracker, GlutenDeltaJobStatisticsTracker}
import org.apache.spark.sql.execution.{SparkPlan, SQLExecution}
import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, FileFormatWriter, WriteJobStatsTracker}
Expand Down Expand Up @@ -72,8 +71,7 @@ class GlutenOptimisticTransaction(delegate: OptimisticTransaction)
// the FileFormatWriter.write call below and will collect per-file stats using
// StatisticsCollection
val optionalStatsTracker =
getOptionalStatsTrackerAndStatsCollection(output, outputPath, partitionSchema, data)._1.map(
new GlutenDeltaJobStatisticsTracker(_))
getOptionalStatsTrackerAndStatsCollection(output, outputPath, partitionSchema, data)._1

val constraints =
Constraints.getAll(metadata, spark) ++ generatedColumnConstraints ++ additionalConstraints
Expand All @@ -87,7 +85,6 @@ class GlutenOptimisticTransaction(delegate: OptimisticTransaction)
statsDataSchema,
trackIdentityHighWaterMarks
)
.map(new GlutenDeltaIdentityColumnStatsTracker(_))

SQLExecution.withNewExecutionId(queryExecution, Option("deltaTransactionalWrite")) {
val outputSpec = FileFormatWriter.OutputSpec(outputPath.toString, Map.empty, output)
Expand All @@ -96,8 +93,8 @@ class GlutenOptimisticTransaction(delegate: OptimisticTransaction)
convertEmptyToNullIfNeeded(queryExecution.executedPlan, partitioningColumns, constraints)
val maybeCheckInvariants = if (constraints.isEmpty) {
// Compared to vanilla Delta, we simply avoid adding the invariant checker
// when the constraint list is empty, to omit the unnecessary transitions
// added around the invariant checker.
// when the constraint list is empty, to prevent the unnecessary transitions
// from being added around the invariant checker.
empty2NullPlan
} else {
DeltaInvariantCheckerExec(empty2NullPlan, constraints)
Expand Down Expand Up @@ -206,7 +203,7 @@ class GlutenOptimisticTransaction(delegate: OptimisticTransaction)
committer.addedStatuses.map {
a =>
a.copy(stats = optionalStatsTracker
.map(_.delegate.recordedStats(a.toPath.getName))
.map(_.recordedStats(a.toPath.getName))
.getOrElse(a.stats))
}
} else {
Expand Down Expand Up @@ -235,7 +232,7 @@ class GlutenOptimisticTransaction(delegate: OptimisticTransaction)
if (resultFiles.nonEmpty && !isOptimize) registerPostCommitHook(AutoCompact)
// Record the updated high water marks to be used during transaction commit.
identityTrackerOpt.ifDefined {
tracker => updatedIdentityHighWaterMarks.appendAll(tracker.delegate.highWaterMarks.toSeq)
tracker => updatedIdentityHighWaterMarks.appendAll(tracker.highWaterMarks.toSeq)
}

resultFiles.toSeq ++ committer.changeFiles
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils}
import org.apache.spark.sql.connector.write.WriterCommitMessage
import org.apache.spark.sql.delta.{DeltaOptions, GlutenParquetFileFormat}
import org.apache.spark.sql.delta.logging.DeltaLogKeys
import org.apache.spark.sql.delta.stats.GlutenDeltaJobStatsTracker
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
Expand Down Expand Up @@ -125,13 +126,14 @@ object GlutenDeltaFileFormatWriter extends LoggingShims {
val dataSchema = dataColumns.toStructType
DataSourceUtils.verifySchema(fileFormat, dataSchema)
DataSourceUtils.checkFieldNames(fileFormat, dataSchema)
// Note: prepareWrite has side effect. It sets "job".
val isNativeWritable = GlutenParquetFileFormat.isNativeWritable(dataSchema)

val outputDataColumns =
if (caseInsensitiveOptions.get(DeltaOptions.WRITE_PARTITION_COLUMNS).contains("true")) {
dataColumns ++ partitionColumns
} else dataColumns

// Note: prepareWrite has side effect. It sets "job".
val outputWriterFactory =
fileFormat.prepareWrite(
sparkSession,
Expand All @@ -140,6 +142,12 @@ object GlutenDeltaFileFormatWriter extends LoggingShims {
outputDataColumns.toStructType
)

val maybeWrappedStatsTrackers: Seq[WriteJobStatsTracker] = if (isNativeWritable) {
statsTrackers.map(GlutenDeltaJobStatsTracker(_))
} else {
statsTrackers
}

val description = new WriteJobDescription(
uuid = UUID.randomUUID.toString,
serializableHadoopConf = new SerializableConfiguration(job.getConfiguration),
Expand All @@ -157,7 +165,7 @@ object GlutenDeltaFileFormatWriter extends LoggingShims {
timeZoneId = caseInsensitiveOptions
.get(DateTimeUtils.TIMEZONE_OPTION)
.getOrElse(sparkSession.sessionState.conf.sessionLocalTimeZone),
statsTrackers = statsTrackers
statsTrackers = maybeWrappedStatsTrackers
)

// We should first sort by dynamic partition columns, then bucket id, and finally sorting
Expand Down Expand Up @@ -222,7 +230,7 @@ object GlutenDeltaFileFormatWriter extends LoggingShims {
partitionColumns,
sortColumns,
orderingMatched,
GlutenParquetFileFormat.isNativeWritable(dataSchema)
isNativeWritable
)
}
}
Expand Down Expand Up @@ -459,6 +467,7 @@ object GlutenDeltaFileFormatWriter extends LoggingShims {
} else {
concurrentOutputWriterSpec match {
case Some(spec) =>
// TODO: Concurrent writer is not yet supported.
new DynamicPartitionDataConcurrentWriter(
description,
taskAttemptContext,
Expand All @@ -468,7 +477,7 @@ object GlutenDeltaFileFormatWriter extends LoggingShims {
case _ =>
// Columnar-based partition writer to divide the input batch by partition values
// and bucket IDs in advance.
new ColumnarDynamicPartitionDataSingleWriter(description, taskAttemptContext, committer)
new GlutenDynamicPartitionDataSingleWriter(description, taskAttemptContext, committer)
}
}

Expand Down Expand Up @@ -525,7 +534,7 @@ object GlutenDeltaFileFormatWriter extends LoggingShims {
}
}

private class ColumnarDynamicPartitionDataSingleWriter(
private class GlutenDynamicPartitionDataSingleWriter(
description: WriteJobDescription,
taskAttemptContext: TaskAttemptContext,
committer: FileCommitProtocol,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,61 +19,42 @@ package org.apache.spark.sql.delta.stats
import org.apache.gluten.execution.{PlaceholderRow, TerminalRow, VeloxColumnarToRowExec}

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.delta.DeltaIdentityColumnStatsTracker
import org.apache.spark.sql.execution.datasources.{WriteJobStatsTracker, WriteTaskStats, WriteTaskStatsTracker}
import org.apache.spark.sql.execution.metric.SQLMetric

class GlutenDeltaJobStatisticsTracker(val delegate: DeltaJobStatisticsTracker)
/**
* A fallback stats tracker where a C2R converter converts all the incoming batches to rows then
* send to the delegate tracker.
*/
private[stats] class GlutenDeltaJobStatsFallbackTracker(val delegate: WriteJobStatsTracker)
extends WriteJobStatsTracker {
import GlutenDeltaJobStatisticsTracker._
import GlutenDeltaJobStatsFallbackTracker._

override def newTaskInstance(): WriteTaskStatsTracker = {
new GlutenDeltaTaskStatisticsTracker(
delegate.newTaskInstance().asInstanceOf[DeltaTaskStatisticsTracker])
new GlutenDeltaTaskStatsFallbackTracker(delegate.newTaskInstance())
}

override def processStats(stats: Seq[WriteTaskStats], jobCommitTime: Long): Unit = {
delegate.processStats(stats, jobCommitTime)
}
}

class GlutenDeltaIdentityColumnStatsTracker(override val delegate: DeltaIdentityColumnStatsTracker)
extends GlutenDeltaJobStatisticsTracker(delegate)

private object GlutenDeltaJobStatisticsTracker {

/**
* This is a temporary implementation of statistics tracker for Delta Lake. It's sub-optimal in
* performance because it internally performs C2R then send rows to the delegate row-based
* tracker.
*
* TODO: Columnar-based statistics collection.
*/
private class GlutenDeltaTaskStatisticsTracker(delegate: DeltaTaskStatisticsTracker)
private object GlutenDeltaJobStatsFallbackTracker {
private class GlutenDeltaTaskStatsFallbackTracker(delegate: WriteTaskStatsTracker)
extends WriteTaskStatsTracker {

private val c2r = new VeloxColumnarToRowExec.Converter(new SQLMetric("convertTime"))

override def newPartition(partitionValues: InternalRow): Unit = {
override def newPartition(partitionValues: InternalRow): Unit =
delegate.newPartition(partitionValues)
}

override def newFile(filePath: String): Unit = {
delegate.newFile(filePath)
}
override def newFile(filePath: String): Unit = delegate.newFile(filePath)

override def closeFile(filePath: String): Unit = {
delegate.closeFile(filePath)
}
override def closeFile(filePath: String): Unit = delegate.closeFile(filePath)

override def newRow(filePath: String, row: InternalRow): Unit = {
row match {
case _: PlaceholderRow =>
case t: TerminalRow =>
c2r.toRowIterator(t.batch()).foreach(eachRow => delegate.newRow(filePath, eachRow))
case otherRow =>
delegate.newRow(filePath, otherRow)
}
override def newRow(filePath: String, row: InternalRow): Unit = row match {
case _: PlaceholderRow =>
case t: TerminalRow =>
c2r.toRowIterator(t.batch()).foreach(eachRow => delegate.newRow(filePath, eachRow))
}

override def getFinalStats(taskCommitTime: Long): WriteTaskStats = {
Expand Down
Original file line number Diff line number Diff line change
@@ -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 org.apache.spark.sql.delta.stats

import org.apache.gluten.execution.{PlaceholderRow, TerminalRow}

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.datasources.{WriteJobStatsTracker, WriteTaskStats, WriteTaskStatsTracker}

/**
* A fallback stats tracker to simply call `newRow` many times when a columnar batch comes. There is
* no C2R process involved to save performance. Therefore, the delegate stats tracker must not read
* the row in its `newRow` implementation.
*/
private[stats] class GlutenDeltaJobStatsRowCountingTracker(val delegate: WriteJobStatsTracker)
extends WriteJobStatsTracker {
import GlutenDeltaJobStatsRowCountingTracker._

override def newTaskInstance(): WriteTaskStatsTracker = {
new GlutenDeltaTaskStatsRowCountingTracker(delegate.newTaskInstance())
}

override def processStats(stats: Seq[WriteTaskStats], jobCommitTime: Long): Unit = {
delegate.processStats(stats, jobCommitTime)
}
}

private object GlutenDeltaJobStatsRowCountingTracker {
private class GlutenDeltaTaskStatsRowCountingTracker(delegate: WriteTaskStatsTracker)
extends WriteTaskStatsTracker {
override def newPartition(partitionValues: InternalRow): Unit =
delegate.newPartition(partitionValues)

override def newFile(filePath: String): Unit = delegate.newFile(filePath)

override def closeFile(filePath: String): Unit = delegate.closeFile(filePath)

override def newRow(filePath: String, row: InternalRow): Unit = row match {
case _: PlaceholderRow =>
case t: TerminalRow =>
for (_ <- 0 until t.batch().numRows()) {
// Here we pass null row to the delegate stats tracker as we assume
// the delegate stats tracker only counts on row numbers.
delegate.newRow(filePath, null)
}
}

override def getFinalStats(taskCommitTime: Long): WriteTaskStats = {
delegate.getFinalStats(taskCommitTime)
}
}
}
Loading