@@ -60,7 +60,14 @@ public class ActionsIterator implements CloseableIterator<ActionWrapper> {
60
60
*/
61
61
private final LinkedList <DeltaLogFile > filesList ;
62
62
63
- private final StructType readSchema ;
63
+ /** Schema used for reading delta files. */
64
+ private final StructType deltaReadSchema ;
65
+
66
+ /**
67
+ * Schema to be used for reading checkpoint files. Checkpoint files can be Parquet or JSON in the
68
+ * case of v2 checkpoints.
69
+ */
70
+ private final StructType checkpointReadSchema ;
64
71
65
72
private final boolean schemaContainsAddOrRemoveFiles ;
66
73
@@ -77,16 +84,26 @@ public class ActionsIterator implements CloseableIterator<ActionWrapper> {
77
84
public ActionsIterator (
78
85
Engine engine ,
79
86
List <FileStatus > files ,
80
- StructType readSchema ,
87
+ StructType deltaReadSchema ,
88
+ Optional <Predicate > checkpointPredicate ) {
89
+ this (engine , files , deltaReadSchema , deltaReadSchema , checkpointPredicate );
90
+ }
91
+
92
+ public ActionsIterator (
93
+ Engine engine ,
94
+ List <FileStatus > files ,
95
+ StructType deltaReadSchema ,
96
+ StructType checkpointReadSchema ,
81
97
Optional <Predicate > checkpointPredicate ) {
82
98
this .engine = engine ;
83
99
this .checkpointPredicate = checkpointPredicate ;
84
100
this .filesList = new LinkedList <>();
85
101
this .filesList .addAll (
86
102
files .stream ().map (DeltaLogFile ::forCommitOrCheckpoint ).collect (Collectors .toList ()));
87
- this .readSchema = readSchema ;
103
+ this .deltaReadSchema = deltaReadSchema ;
104
+ this .checkpointReadSchema = checkpointReadSchema ;
88
105
this .actionsIter = Optional .empty ();
89
- this .schemaContainsAddOrRemoveFiles = LogReplay .containsAddOrRemoveFileActions (readSchema );
106
+ this .schemaContainsAddOrRemoveFiles = LogReplay .containsAddOrRemoveFileActions (deltaReadSchema );
90
107
}
91
108
92
109
@ Override
@@ -105,7 +122,7 @@ public boolean hasNext() {
105
122
106
123
/**
107
124
* @return a tuple of (ColumnarBatch, isFromCheckpoint), where ColumnarBatch conforms to the
108
- * instance {@link #readSchema }.
125
+ * instance {@link #deltaReadSchema }.
109
126
*/
110
127
@ Override
111
128
public ActionWrapper next () {
@@ -176,9 +193,9 @@ private CloseableIterator<ColumnarBatch> getActionsIterFromSinglePartOrV2Checkpo
176
193
FileStatus file , String fileName ) throws IOException {
177
194
// If the sidecars may contain the current action, read sidecars from the top-level v2
178
195
// checkpoint file(to be read later).
179
- StructType modifiedReadSchema = readSchema ;
196
+ StructType modifiedReadSchema = checkpointReadSchema ;
180
197
if (schemaContainsAddOrRemoveFiles ) {
181
- modifiedReadSchema = LogReplay .withSidecarFileSchema (readSchema );
198
+ modifiedReadSchema = LogReplay .withSidecarFileSchema (checkpointReadSchema );
182
199
}
183
200
184
201
long checkpointVersion = checkpointVersion (file .getPath ());
@@ -195,7 +212,8 @@ private CloseableIterator<ColumnarBatch> getActionsIterFromSinglePartOrV2Checkpo
195
212
checkpointPredicateIncludingSidecars = checkpointPredicate ;
196
213
}
197
214
final CloseableIterator <ColumnarBatch > topLevelIter ;
198
- StructType finalModifiedReadSchema = modifiedReadSchema ;
215
+ StructType finalReadSchema = modifiedReadSchema ;
216
+
199
217
if (fileName .endsWith (".parquet" )) {
200
218
topLevelIter =
201
219
wrapEngineExceptionThrowsIO (
@@ -204,11 +222,11 @@ private CloseableIterator<ColumnarBatch> getActionsIterFromSinglePartOrV2Checkpo
204
222
.getParquetHandler ()
205
223
.readParquetFiles (
206
224
singletonCloseableIterator (file ),
207
- finalModifiedReadSchema ,
225
+ finalReadSchema ,
208
226
checkpointPredicateIncludingSidecars ),
209
227
"Reading parquet log file `%s` with readSchema=%s and predicate=%s" ,
210
228
file ,
211
- modifiedReadSchema ,
229
+ finalReadSchema ,
212
230
checkpointPredicateIncludingSidecars );
213
231
} else if (fileName .endsWith (".json" )) {
214
232
topLevelIter =
@@ -218,11 +236,11 @@ private CloseableIterator<ColumnarBatch> getActionsIterFromSinglePartOrV2Checkpo
218
236
.getJsonHandler ()
219
237
.readJsonFiles (
220
238
singletonCloseableIterator (file ),
221
- finalModifiedReadSchema ,
239
+ finalReadSchema ,
222
240
checkpointPredicateIncludingSidecars ),
223
241
"Reading JSON log file `%s` with readSchema=%s and predicate=%s" ,
224
242
file ,
225
- modifiedReadSchema ,
243
+ finalReadSchema ,
226
244
checkpointPredicateIncludingSidecars );
227
245
} else {
228
246
throw new IOException ("Unrecognized top level v2 checkpoint file format: " + fileName );
@@ -309,10 +327,12 @@ private CloseableIterator<ActionWrapper> getNextActionsIter() {
309
327
engine
310
328
.getJsonHandler ()
311
329
.readJsonFiles (
312
- singletonCloseableIterator (nextFile ), readSchema , Optional .empty ()),
330
+ singletonCloseableIterator (nextFile ),
331
+ deltaReadSchema ,
332
+ Optional .empty ()),
313
333
"Reading JSON log file `%s` with readSchema=%s" ,
314
334
nextFile ,
315
- readSchema );
335
+ deltaReadSchema );
316
336
317
337
return combine (
318
338
dataIter ,
@@ -344,10 +364,11 @@ private CloseableIterator<ActionWrapper> getNextActionsIter() {
344
364
() ->
345
365
engine
346
366
.getParquetHandler ()
347
- .readParquetFiles (checkpointFiles , readSchema , checkpointPredicate ),
367
+ .readParquetFiles (
368
+ checkpointFiles , deltaReadSchema , checkpointPredicate ),
348
369
"Reading checkpoint sidecars [%s] with readSchema=%s and predicate=%s" ,
349
370
checkpointFiles ,
350
- readSchema ,
371
+ deltaReadSchema ,
351
372
checkpointPredicate );
352
373
353
374
long version = checkpointVersion (nextFilePath );
0 commit comments