Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Refactoring and optimisation of RestoreTableCommand #912

Closed
Closed
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -17,9 +17,8 @@
package io.delta.tables.execution

import scala.collection.Map

import org.apache.spark.sql.delta.{DeltaLog, RestoreTableCommand}
import org.apache.spark.sql.delta.commands.{DeltaGenerateCommand, VacuumCommand}
import org.apache.spark.sql.delta.DeltaLog
import org.apache.spark.sql.delta.commands.{DeltaGenerateCommand, RestoreTableCommand, VacuumCommand}
import org.apache.spark.sql.delta.util.AnalysisHelper
import io.delta.tables.DeltaTable

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
* limitations under the License.
*/

package org.apache.spark.sql.delta
package org.apache.spark.sql.delta.commands

import java.sql.Timestamp

Expand All @@ -23,12 +23,12 @@ import scala.util.{Success, Try}

import org.apache.spark.sql.delta.DeltaErrors.timestampInvalid
import org.apache.spark.sql.delta.actions.{AddFile, RemoveFile}
import org.apache.spark.sql.delta.commands.DeltaCommand
import org.apache.spark.sql.delta.util.DeltaFileOperations.absolutePath

import org.apache.spark.sql.{Dataset, Row, SparkSession}
import org.apache.spark.sql.catalyst.expressions.Literal
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.delta.{DeltaErrors, DeltaLog, DeltaOperations, Snapshot}
import org.apache.spark.sql.execution.command.LeafRunnableCommand
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.IGNORE_MISSING_FILES
Expand Down Expand Up @@ -91,9 +91,6 @@ case class RestoreTableCommand(
"left_anti")
.as[AddFile]
.map(_.copy(dataChange = true))
// To avoid recompute of Dataset with wide transformation by toLocalIterator and
// checkSnapshotFilesAvailability method with spark.sql.files.ignoreMissingFiles=false
.cache()

val filesToRemove = latestSnapshotFiles
.join(
Expand All @@ -102,8 +99,6 @@ case class RestoreTableCommand(
"left_anti")
.as[AddFile]
.map(_.removeWithTimestamp())
// To avoid recompute of Dataset with wide transformation by toLocalIterator
.cache()

try {
checkSnapshotFilesAvailability(deltaLog, filesToAdd, versionToRestore)
Expand All @@ -124,7 +119,6 @@ case class RestoreTableCommand(
metrics)
} finally {
filesToAdd.unpersist()
Maks-D marked this conversation as resolved.
Show resolved Hide resolved
filesToRemove.unpersist()
}
}

Expand Down Expand Up @@ -184,6 +178,8 @@ case class RestoreTableCommand(
.getConf(IGNORE_MISSING_FILES)

if (!ignore) {
// To avoid recompute of files Dataset in calling method
files.cache()
val path = deltaLog.dataPath
val hadoopConf = spark.sparkContext.broadcast(
new SerializableConfiguration(deltaLog.newDeltaHadoopConf()))
Expand Down