@@ -23,13 +23,15 @@ import org.apache.spark.sql.delta.DeltaConfigs.{MANAGED_COMMIT_OWNER_CONF, MANAG
23
23
import org .apache .spark .sql .delta .DeltaLog
24
24
import org .apache .spark .sql .delta .DeltaTestUtils .createTestAddFile
25
25
import org .apache .spark .sql .delta .InitialSnapshot
26
+ import org .apache .spark .sql .delta .Snapshot
26
27
import org .apache .spark .sql .delta .actions .{Action , Metadata }
27
28
import org .apache .spark .sql .delta .sources .DeltaSQLConf
28
29
import org .apache .spark .sql .delta .storage .LogStore
29
30
import org .apache .spark .sql .delta .test .DeltaSQLCommandTest
30
31
import org .apache .spark .sql .delta .test .DeltaSQLTestUtils
31
32
import org .apache .spark .sql .delta .test .DeltaTestImplicits ._
32
33
import org .apache .spark .sql .delta .util .{FileNames , JsonUtils }
34
+ import org .apache .spark .sql .delta .util .FileNames .{CompactedDeltaFile , DeltaFile }
33
35
import org .apache .hadoop .conf .Configuration
34
36
import org .apache .hadoop .fs .{FileStatus , Path }
35
37
@@ -145,50 +147,41 @@ class ManagedCommitSuite
145
147
}
146
148
147
149
// Test commit owner changed on concurrent cluster
148
- test (" snapshot is updated recursively when FS table is converted to commit owner " +
149
- " table on a concurrent cluster" ) {
150
+ testWithoutManagedCommits (" snapshot is updated recursively when FS table is converted to commit" +
151
+ " owner table on a concurrent cluster" ) {
150
152
val commitStore = new TrackingCommitStore (new InMemoryCommitStore (batchSize = 10 ))
151
153
val builder = TrackingInMemoryCommitStoreBuilder (batchSize = 10 , Some (commitStore))
152
154
CommitStoreProvider .registerBuilder(builder)
153
- val oldCommitOwnerValue = spark.conf.get(MANAGED_COMMIT_OWNER_NAME .defaultTablePropertyKey)
154
- spark.conf.unset(MANAGED_COMMIT_OWNER_NAME .defaultTablePropertyKey)
155
155
156
- try {
157
- withTempDir { tempDir =>
158
- val tablePath = tempDir.getAbsolutePath
159
- val deltaLog1 = DeltaLog .forTable(spark, tablePath)
160
- deltaLog1.startTransaction().commitManually(Metadata ())
161
- deltaLog1.startTransaction().commitManually(createTestAddFile(" f1" ))
162
- deltaLog1.startTransaction().commitManually()
163
- val snapshotV2 = deltaLog1.update()
164
- assert(snapshotV2.version === 2 )
165
- assert(snapshotV2.commitStoreOpt.isEmpty)
166
- DeltaLog .clearCache()
156
+ withTempDir { tempDir =>
157
+ val tablePath = tempDir.getAbsolutePath
158
+ val deltaLog1 = DeltaLog .forTable(spark, tablePath)
159
+ deltaLog1.startTransaction().commitManually(Metadata ())
160
+ deltaLog1.startTransaction().commitManually(createTestAddFile(" f1" ))
161
+ deltaLog1.startTransaction().commitManually()
162
+ val snapshotV2 = deltaLog1.update()
163
+ assert(snapshotV2.version === 2 )
164
+ assert(snapshotV2.commitStoreOpt.isEmpty)
165
+ DeltaLog .clearCache()
167
166
168
- // Add new commit to convert FS table to managed-commit table
169
- val deltaLog2 = DeltaLog .forTable(spark, tablePath)
170
- val oldMetadata = snapshotV2.metadata
171
- val commitOwner = (MANAGED_COMMIT_OWNER_NAME .key -> " tracking-in-memory" )
172
- val newMetadata = oldMetadata.copy(configuration = oldMetadata.configuration + commitOwner)
173
- deltaLog2.startTransaction().commitManually(newMetadata)
174
- commitStore.registerTable(deltaLog2.logPath, 3 )
175
- deltaLog2.startTransaction().commitManually(createTestAddFile(" f2" ))
176
- deltaLog2.startTransaction().commitManually()
177
- val snapshotV5 = deltaLog2.unsafeVolatileSnapshot
178
- assert(snapshotV5.version === 5 )
179
- assert(snapshotV5.commitStoreOpt.nonEmpty)
180
- // only delta 4/5 will be un-backfilled and should have two dots in filename (x.uuid.json)
181
- assert(snapshotV5.logSegment.deltas.count(_.getPath.getName.count(_ == '.' ) == 2 ) === 2 )
182
-
183
- val usageRecords = Log4jUsageLogger .track {
184
- val newSnapshotV5 = deltaLog1.update()
185
- assert(newSnapshotV5.version === 5 )
186
- assert(newSnapshotV5.logSegment.deltas === snapshotV5.logSegment.deltas)
187
- }
188
- assert(filterUsageRecords(usageRecords, " delta.readChecksum" ).size === 2 )
167
+ // Add new commit to convert FS table to managed-commit table
168
+ val deltaLog2 = DeltaLog .forTable(spark, tablePath)
169
+ enableManagedCommit(deltaLog2, commitOwner = " tracking-in-memory" )
170
+ commitStore.registerTable(deltaLog2.logPath, 3 )
171
+ deltaLog2.startTransaction().commitManually(createTestAddFile(" f2" ))
172
+ deltaLog2.startTransaction().commitManually()
173
+ val snapshotV5 = deltaLog2.unsafeVolatileSnapshot
174
+ assert(snapshotV5.version === 5 )
175
+ assert(snapshotV5.commitStoreOpt.nonEmpty)
176
+ // only delta 4/5 will be un-backfilled and should have two dots in filename (x.uuid.json)
177
+ assert(snapshotV5.logSegment.deltas.count(_.getPath.getName.count(_ == '.' ) == 2 ) === 2 )
178
+
179
+ val usageRecords = Log4jUsageLogger .track {
180
+ val newSnapshotV5 = deltaLog1.update()
181
+ assert(newSnapshotV5.version === 5 )
182
+ assert(newSnapshotV5.logSegment.deltas === snapshotV5.logSegment.deltas)
189
183
}
190
- } finally {
191
- spark.conf.set(MANAGED_COMMIT_OWNER_NAME .defaultTablePropertyKey, oldCommitOwnerValue)
184
+ assert(filterUsageRecords(usageRecords, " delta.readChecksum" ).size === 2 )
192
185
}
193
186
}
194
187
@@ -517,7 +510,7 @@ class ManagedCommitSuite
517
510
}
518
511
}
519
512
520
- testWithDifferentBackfillInterval(" ensure backfills commit files works as expected " ) { _ =>
513
+ testWithDifferentBackfillInterval(" Snapshot.ensureCommitFilesBackfilled " ) { _ =>
521
514
withTempDir { tempDir =>
522
515
val tablePath = tempDir.getAbsolutePath
523
516
@@ -534,6 +527,113 @@ class ManagedCommitSuite
534
527
val backfilledCommitFiles = (0 to 9 ).map(
535
528
version => FileNames .unsafeDeltaFile(log.logPath, version))
536
529
assert(commitFiles.toSeq == backfilledCommitFiles)
537
- }
530
+ }
531
+ }
532
+
533
+ testWithoutManagedCommits(" DeltaLog.getSnapshotAt" ) {
534
+ val commitStore = new TrackingCommitStore (new InMemoryCommitStore (batchSize = 10 ))
535
+ val builder = TrackingInMemoryCommitStoreBuilder (batchSize = 10 , Some (commitStore))
536
+ CommitStoreProvider .registerBuilder(builder)
537
+ def checkGetSnapshotAt (
538
+ deltaLog : DeltaLog ,
539
+ version : Long ,
540
+ expectedUpdateCount : Int ,
541
+ expectedListingCount : Int ): Snapshot = {
542
+ var snapshot : Snapshot = null
543
+
544
+ val usageRecords = Log4jUsageLogger .track {
545
+ snapshot = deltaLog.getSnapshotAt(version)
546
+ assert(snapshot.version === version)
547
+ }
548
+ assert(filterUsageRecords(usageRecords, " deltaLog.update" ).size === expectedUpdateCount)
549
+ // deltaLog.update() will internally do listing
550
+ assert(filterUsageRecords(usageRecords, " delta.deltaLog.listDeltaAndCheckpointFiles" ).size
551
+ === expectedListingCount)
552
+ val versionsInLogSegment = if (version < 6 ) {
553
+ snapshot.logSegment.deltas.map(FileNames .deltaVersion(_))
554
+ } else {
555
+ snapshot.logSegment.deltas.flatMap {
556
+ case DeltaFile (_, deltaVersion) => Seq (deltaVersion)
557
+ case CompactedDeltaFile (_, startVersion, endVersion) => (startVersion to endVersion)
558
+ }
559
+ }
560
+ assert(versionsInLogSegment === (0L to version))
561
+ snapshot
562
+ }
563
+
564
+ withTempDir { dir =>
565
+ val tablePath = dir.getAbsolutePath
566
+ // Part-1: Validate getSnapshotAt API works as expected for non-managed commit tables
567
+ // commit 0, 1, 2 on FS table
568
+ Seq (1 ).toDF.write.format(" delta" ).mode(" overwrite" ).save(tablePath) // v0
569
+ Seq (1 ).toDF.write.format(" delta" ).mode(" overwrite" ).save(tablePath) // v1
570
+ val deltaLog1 = DeltaLog .forTable(spark, tablePath)
571
+ DeltaLog .clearCache()
572
+ Seq (1 ).toDF.write.format(" delta" ).mode(" overwrite" ).save(tablePath) // v2
573
+ assert(deltaLog1.unsafeVolatileSnapshot.version === 1 )
574
+
575
+ checkGetSnapshotAt(deltaLog1, version = 1 , expectedUpdateCount = 0 , expectedListingCount = 0 )
576
+ // deltaLog1 still points to version 1. So, we will do listing to get v0.
577
+ checkGetSnapshotAt(deltaLog1, version = 0 , expectedUpdateCount = 0 , expectedListingCount = 1 )
578
+ // deltaLog1 still points to version 1 although we are asking for v2 So we do a
579
+ // deltaLog.update - the update will internally do listing.Since the updated snapshot is same
580
+ // as what we want, so we won't create another snapshot and do another listing.
581
+ checkGetSnapshotAt(deltaLog1, version = 2 , expectedUpdateCount = 1 , expectedListingCount = 1 )
582
+ var deltaLog2 = DeltaLog .forTable(spark, tablePath)
583
+ Seq (deltaLog1, deltaLog2).foreach { log => assert(log.unsafeVolatileSnapshot.version === 2 ) }
584
+ DeltaLog .clearCache()
585
+
586
+ // Part-2: Validate getSnapshotAt API works as expected for managed commit tables when the
587
+ // switch is made
588
+ // commit 3
589
+ enableManagedCommit(DeltaLog .forTable(spark, tablePath), " tracking-in-memory" )
590
+ commitStore.registerTable(deltaLog1.logPath, maxCommitVersion = 3 )
591
+ // commit 4
592
+ Seq (1 ).toDF.write.format(" delta" ).mode(" overwrite" ).save(tablePath)
593
+ // the old deltaLog objects still points to version 2
594
+ Seq (deltaLog1, deltaLog2).foreach { log => assert(log.unsafeVolatileSnapshot.version === 2 ) }
595
+ // deltaLog1 points to version 2. So, we will do listing to get v1. Snapshot update not
596
+ // needed as what we are looking for is less than what deltaLog1 points to.
597
+ checkGetSnapshotAt(deltaLog1, version = 1 , expectedUpdateCount = 0 , expectedListingCount = 1 )
598
+ // deltaLog1.unsafeVolatileSnapshot.version points to v2 - return it directly.
599
+ checkGetSnapshotAt(deltaLog1, version = 2 , expectedUpdateCount = 0 , expectedListingCount = 0 )
600
+ // We are asking for v3 although the deltaLog1.unsafeVolatileSnapshot is for v2. So this will
601
+ // need deltaLog.update() to get the latest snapshot first - this update itself internally
602
+ // will do 2 round of listing as we are discovering a commit store after first round of
603
+ // listing. Once the update finishes, deltaLog1 will point to v4. So we need another round of
604
+ // listing to get just v3.
605
+ checkGetSnapshotAt(deltaLog1, version = 3 , expectedUpdateCount = 1 , expectedListingCount = 3 )
606
+ // Ask for v3 again - this time deltaLog1.unsafeVolatileSnapshot points to v4.
607
+ // So we don't need deltaLog.update as version which we are asking is less than pinned
608
+ // version. Just do listing and get the snapshot.
609
+ checkGetSnapshotAt(deltaLog1, version = 3 , expectedUpdateCount = 0 , expectedListingCount = 1 )
610
+ // deltaLog1.unsafeVolatileSnapshot.version points to v4 - return it directly.
611
+ checkGetSnapshotAt(deltaLog1, version = 4 , expectedUpdateCount = 0 , expectedListingCount = 0 )
612
+ // We are asking for v3 although the deltaLog2.unsafeVolatileSnapshot is for v2. So this will
613
+ // need deltaLog.update() to get the latest snapshot first - this update itself internally
614
+ // will do 2 round of listing as we are discovering a commit store after first round of
615
+ // listing. Once the update finishes, deltaLog2 will point to v4. It can be returned directly.
616
+ checkGetSnapshotAt(deltaLog2, version = 4 , expectedUpdateCount = 1 , expectedListingCount = 2 )
617
+
618
+ // Part-2: Validate getSnapshotAt API works as expected for managed commit tables
619
+ Seq (1 ).toDF.write.format(" delta" ).mode(" overwrite" ).save(tablePath) // v5
620
+ deltaLog2 = DeltaLog .forTable(spark, tablePath)
621
+ DeltaLog .clearCache()
622
+ Seq (1 ).toDF.write.format(" delta" ).mode(" overwrite" ).save(tablePath) // v6
623
+ Seq (1 ).toDF.write.format(" delta" ).mode(" overwrite" ).save(tablePath) // v7
624
+ assert(deltaLog2.unsafeVolatileSnapshot.version === 5 )
625
+ checkGetSnapshotAt(deltaLog2, version = 1 , expectedUpdateCount = 0 , expectedListingCount = 1 )
626
+ checkGetSnapshotAt(deltaLog2, version = 2 , expectedUpdateCount = 0 , expectedListingCount = 1 )
627
+ checkGetSnapshotAt(deltaLog2, version = 4 , expectedUpdateCount = 0 , expectedListingCount = 1 )
628
+ checkGetSnapshotAt(deltaLog2, version = 5 , expectedUpdateCount = 0 , expectedListingCount = 0 )
629
+ checkGetSnapshotAt(deltaLog2, version = 6 , expectedUpdateCount = 1 , expectedListingCount = 2 )
630
+ }
631
+ }
632
+
633
+ private def enableManagedCommit (deltaLog : DeltaLog , commitOwner : String ): Unit = {
634
+ val oldMetadata = deltaLog.update().metadata
635
+ val commitOwnerConf = (MANAGED_COMMIT_OWNER_NAME .key -> commitOwner)
636
+ val newMetadata = oldMetadata.copy(configuration = oldMetadata.configuration + commitOwnerConf)
637
+ deltaLog.startTransaction().commitManually(newMetadata)
538
638
}
539
639
}
0 commit comments