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

Support checkpoints on Delta tables with deletion vectors #1576

Closed
wants to merge 2 commits into from
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 @@ -723,7 +723,8 @@ object Checkpoints extends DeltaLogging {
col("add.size"),
col("add.modificationTime"),
col("add.dataChange"), // actually not really useful here
col("add.tags")) ++
col("add.tags"),
col("add.deletionVector")) ++
additionalCols: _*
))
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,13 @@ import java.net.URI

// scalastyle:off import.ordering.noEmptyLine
import org.apache.spark.sql.delta.actions.AddCDCFile
import org.apache.spark.sql.delta.deletionvectors.DeletionVectorsSuite
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.storage.LocalLogStore
import org.apache.spark.sql.delta.test.DeltaSQLCommandTest
import org.apache.spark.sql.delta.test.DeltaTestImplicits._
import org.apache.spark.sql.delta.util.FileNames
import org.apache.commons.io.FileUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FSDataOutputStream, Path, RawLocalFileSystem}
import org.apache.hadoop.fs.permission.FsPermission
Expand Down Expand Up @@ -260,6 +262,39 @@ class CheckpointsSuite extends QueryTest
Seq(0, 0, 1, 1, 2, 2, 3, 3, 4, 4).map { i => Row(i) })
}
}

test("checkpoint with DVs") {
withTempDir { tempDir =>
val source = new File(DeletionVectorsSuite.table1Path) // this table has DVs in two versions
val target = new File(tempDir, "insertTest")

// Copy the source2 DV table to a temporary directory, so that we do updates to it
FileUtils.copyDirectory(source, target)

def insertData(data: String): Unit = {
spark.sql(s"INSERT INTO TABLE delta.`${target.getAbsolutePath}` $data")
}
val newData = Seq.range(3000, 3010)
newData.foreach { i => insertData(s"VALUES($i)") }

// Check the target file has checkpoint generated
val deltaLog = DeltaLog.forTable(spark, target.getAbsolutePath)
verifyCheckpoint(deltaLog.readLastCheckpointFile(), 10, None)

// Delete the commit files 0-9, so that we are forced to read the checkpoint file
val logPath = new Path(new File(target, "_delta_log").getAbsolutePath)
for (i <- 0 to 10) {
val file = new File(FileNames.deltaFile(logPath, version = i).toString)
file.delete()
}

// Make sure the contents are the same
import testImplicits._
checkAnswer(
spark.sql(s"SELECT * FROM delta.`${target.getAbsolutePath}`"),
(DeletionVectorsSuite.expectedTable1DataV4 ++ newData).toSeq.toDF())
}
}
}

/**
Expand Down