3636import org .apache .iceberg .types .Types ;
3737import org .apache .iceberg .util .ParallelIterable ;
3838
39- public class RestTableScan extends DataTableScan {
39+ class RestTableScan extends DataTableScan {
4040 private final RESTClient client ;
4141 private final String path ;
4242 private final Supplier <Map <String , String >> headers ;
@@ -48,7 +48,7 @@ public class RestTableScan extends DataTableScan {
4848 // TODO revisit if this property should be configurable
4949 private static final int FETCH_PLANNING_SLEEP_DURATION_MS = 1000 ;
5050
51- public RestTableScan (
51+ RestTableScan (
5252 Table table ,
5353 Schema schema ,
5454 TableScanContext context ,
@@ -85,6 +85,9 @@ protected TableScan newRefinedScan(
8585
8686 @ Override
8787 public CloseableIterable <FileScanTask > planFiles () {
88+ Long startSnapshotId = context ().fromSnapshotId ();
89+ Long endSnapshotId = context ().toSnapshotId ();
90+ Long snapshotId = snapshotId ();
8891 List <String > selectedColumns =
8992 schema ().columns ().stream ().map (Types .NestedField ::name ).collect (Collectors .toList ());
9093
@@ -96,10 +99,6 @@ public CloseableIterable<FileScanTask> planFiles() {
9699 .collect (Collectors .toList ());
97100 }
98101
99- Long startSnapshotId = context ().fromSnapshotId ();
100- Long endSnapshotId = context ().toSnapshotId ();
101- Long snapshotId = snapshotId ();
102-
103102 PlanTableScanRequest .Builder planTableScanRequestBuilder =
104103 new PlanTableScanRequest .Builder ()
105104 .withSelect (selectedColumns )
@@ -127,12 +126,12 @@ public CloseableIterable<FileScanTask> planFiles() {
127126 }
128127
129128 private CloseableIterable <FileScanTask > planTableScan (PlanTableScanRequest planTableScanRequest ) {
130- // we need to inject specById map here and also the caseSensitive
131129 ParserContext context =
132130 ParserContext .builder ()
133131 .add ("specsById" , table .specs ())
134132 .add ("caseSensitive" , context ().caseSensitive ())
135133 .build ();
134+
136135 PlanTableScanResponse response =
137136 client .post (
138137 resourcePaths .planTableScan (tableIdentifier ),
@@ -146,13 +145,15 @@ private CloseableIterable<FileScanTask> planTableScan(PlanTableScanRequest planT
146145 PlanStatus planStatus = response .planStatus ();
147146 switch (planStatus ) {
148147 case COMPLETED :
149- // List<FileScanTask> fileScanTasks = bindFileScanTasksWithSpec(response.fileScanTasks());
150148 return getScanTasksIterable (response .planTasks (), response .fileScanTasks ());
151149 case SUBMITTED :
152150 return fetchPlanningResult (response .planId ());
153151 case FAILED :
154- throw new RuntimeException (
152+ throw new IllegalStateException (
155153 "Received \" failed\" status from service when planning a table scan" );
154+ case CANCELLED :
155+ throw new IllegalStateException (
156+ "Received \" cancelled\" status from service when planning a table scan" );
156157 default :
157158 throw new RuntimeException (
158159 String .format ("Invalid planStatus during planTableScan: %s" , planStatus ));
@@ -181,9 +182,10 @@ private CloseableIterable<FileScanTask> fetchPlanningResult(String planId) {
181182 PlanStatus planStatus = response .planStatus ();
182183 switch (planStatus ) {
183184 case COMPLETED :
184- // List<FileScanTask> fileScanTasks = bindFileScanTasksWithSpec(response.fileScanTasks());
185185 return getScanTasksIterable (response .planTasks (), response .fileScanTasks ());
186186 case SUBMITTED :
187+ // TODO: Think more about whether we should use a backoff strategy here.
188+ // For now, we will just sleep for a fixed duration before checking the status again.
187189 try {
188190 Thread .sleep (FETCH_PLANNING_SLEEP_DURATION_MS );
189191 } catch (InterruptedException e ) {
0 commit comments