@@ -154,7 +154,11 @@ private static class InitialSnapshotCache {
154154 private final AtomicReference <InitialSnapshotCache > cachedInitialSnapshot =
155155 new AtomicReference <>(null );
156156
157+ private final AtomicReference <DataFrameSnapshotCache > cachedDataFrameSnapshot =
158+ new AtomicReference <>(null );
159+
157160 private final int maxInitialSnapshotFiles ;
161+ private final boolean useDataFrameBasedInitialSnapshot ;
158162
159163 public SparkMicroBatchStream (
160164 DeltaSnapshotManager snapshotManager ,
@@ -204,6 +208,12 @@ public SparkMicroBatchStream(
204208 .sessionState ()
205209 .conf ()
206210 .getConf (DeltaSQLConf .DELTA_STREAMING_INITIAL_SNAPSHOT_MAX_FILES ());
211+ this .useDataFrameBasedInitialSnapshot =
212+ (Boolean )
213+ spark
214+ .sessionState ()
215+ .conf ()
216+ .getConf (DeltaSQLConf .DELTA_STREAMING_USE_DATAFRAME_INITIAL_SNAPSHOT ());
207217
208218 boolean isStreamingFromColumnMappingTable =
209219 ColumnMapping .getColumnMappingMode (
@@ -450,6 +460,14 @@ public void commit(Offset end) {
450460 @ Override
451461 public void stop () {
452462 cachedInitialSnapshot .set (null );
463+ invalidateDataFrameCache ();
464+ }
465+
466+ private void invalidateDataFrameCache () {
467+ DataFrameSnapshotCache prev = cachedDataFrameSnapshot .getAndSet (null );
468+ if (prev != null ) {
469+ prev .close ();
470+ }
453471 }
454472
455473 ///////////////////////
@@ -1042,6 +1060,10 @@ private long addIndexedFilesAndReturnNextIndex(
10421060 * @return An iterator of IndexedFile representing the snapshot files
10431061 */
10441062 private CloseableIterator <IndexedFile > getSnapshotFiles (long version ) {
1063+ if (useDataFrameBasedInitialSnapshot ) {
1064+ return getSnapshotFilesViaDataFrame (version );
1065+ }
1066+
10451067 InitialSnapshotCache cache = cachedInitialSnapshot .get ();
10461068
10471069 if (cache != null && cache .version != null && cache .version == version ) {
@@ -1056,6 +1078,78 @@ private CloseableIterator<IndexedFile> getSnapshotFiles(long version) {
10561078 return Utils .toCloseableIterator (indexedFiles .iterator ());
10571079 }
10581080
1081+ private CloseableIterator <IndexedFile > getSnapshotFilesViaDataFrame (long version ) {
1082+ DataFrameSnapshotCache dfCache = cachedDataFrameSnapshot .get ();
1083+ if (dfCache != null && dfCache .getVersion () == version ) {
1084+ return dataFrameToIndexedFiles (dfCache .getSortedAddFiles (), version );
1085+ }
1086+
1087+ invalidateDataFrameCache ();
1088+
1089+ SnapshotImpl snapshot = (SnapshotImpl ) snapshotManager .loadSnapshotAt (version );
1090+ io .delta .spark .internal .v2 .utils .SerializableReadOnlySnapshot serSnapshot =
1091+ io .delta .spark .internal .v2 .utils .SerializableReadOnlySnapshot .fromSnapshot (
1092+ snapshot , hadoopConf );
1093+
1094+ ScanFileRDD rdd = new ScanFileRDD (spark .sparkContext (), serSnapshot );
1095+ org .apache .spark .sql .Dataset <org .apache .spark .sql .Row > df =
1096+ spark
1097+ .createDataFrame (rdd , ScanFileRDD .SPARK_SCHEMA )
1098+ .orderBy ("modificationTime" , "path" )
1099+ .persist (org .apache .spark .storage .StorageLevel .MEMORY_AND_DISK ());
1100+
1101+ dfCache = new DataFrameSnapshotCache (version , df );
1102+ cachedDataFrameSnapshot .set (dfCache );
1103+
1104+ return dataFrameToIndexedFiles (df , version );
1105+ }
1106+
1107+ /**
1108+ * Converts a sorted DataFrame of AddFile rows into a lazy CloseableIterator of IndexedFiles,
1109+ * wrapped with BEGIN/END sentinels. Uses toLocalIterator() to stream rows from executors to the
1110+ * driver one at a time, avoiding pulling all data into driver memory.
1111+ */
1112+ private static CloseableIterator <IndexedFile > dataFrameToIndexedFiles (
1113+ org .apache .spark .sql .Dataset <org .apache .spark .sql .Row > df , long version ) {
1114+
1115+ java .util .Iterator <org .apache .spark .sql .Row > localIter = df .toLocalIterator ();
1116+
1117+ return new CloseableIterator <IndexedFile >() {
1118+ private boolean sentBegin = false ;
1119+ private boolean sentEnd = false ;
1120+ private long index = 0 ;
1121+
1122+ @ Override
1123+ public boolean hasNext () {
1124+ return !sentEnd ;
1125+ }
1126+
1127+ @ Override
1128+ public IndexedFile next () {
1129+ if (!sentBegin ) {
1130+ sentBegin = true ;
1131+ return new IndexedFile (version , DeltaSourceOffset .BASE_INDEX (), null );
1132+ }
1133+
1134+ if (localIter .hasNext ()) {
1135+ org .apache .spark .sql .Row sparkRow = localIter .next ();
1136+ io .delta .kernel .data .Row kernelRow =
1137+ new io .delta .spark .internal .v2 .utils .SparkRowToKernelRow (
1138+ sparkRow , AddFile .SCHEMA_WITHOUT_STATS );
1139+ return new IndexedFile (version , index ++, new AddFile (kernelRow ));
1140+ }
1141+
1142+ sentEnd = true ;
1143+ return new IndexedFile (version , DeltaSourceOffset .END_INDEX (), null );
1144+ }
1145+
1146+ @ Override
1147+ public void close () throws IOException {
1148+ // toLocalIterator() resources are managed by Spark
1149+ }
1150+ };
1151+ }
1152+
10591153 /** Loads snapshot files at the specified version. */
10601154 private List <IndexedFile > loadAndValidateSnapshot (long version ) {
10611155 Snapshot snapshot = snapshotManager .loadSnapshotAt (version );
0 commit comments