2525import org .apache .iceberg .FileScanTask ;
2626import org .apache .iceberg .exceptions .NoSuchPlanIdException ;
2727import org .apache .iceberg .exceptions .NoSuchPlanTaskException ;
28+ import org .apache .iceberg .relocated .com .google .common .base .Preconditions ;
2829import org .apache .iceberg .relocated .com .google .common .base .Splitter ;
2930import org .apache .iceberg .relocated .com .google .common .collect .ImmutableList ;
3031import org .apache .iceberg .relocated .com .google .common .collect .Iterables ;
@@ -43,10 +44,12 @@ class InMemoryPlanningState {
4344
4445 private final Map <String , List <FileScanTask >> planTaskToFileScanTasks ;
4546 private final Map <String , String > planTaskToNext ;
47+ private final Map <String , PlanStatus > asyncPlanningStates ;
4648
4749 private InMemoryPlanningState () {
4850 this .planTaskToFileScanTasks = Maps .newConcurrentMap ();
4951 this .planTaskToNext = Maps .newConcurrentMap ();
52+ this .asyncPlanningStates = Maps .newConcurrentMap ();
5053 }
5154
5255 static InMemoryPlanningState getInstance () {
@@ -68,6 +71,44 @@ void addNextPlanTask(String currentTask, String nextTask) {
6871 planTaskToNext .put (currentTask , nextTask );
6972 }
7073
74+ void addAsyncPlan (String plan ) {
75+ PlanStatus existingStatus = asyncPlanningStates .get (plan );
76+ Preconditions .checkArgument (
77+ existingStatus == null , "Plan %s already exists with status %s" , plan , existingStatus );
78+ asyncPlanningStates .put (plan , PlanStatus .SUBMITTED );
79+ }
80+
81+ PlanStatus asyncPlanStatus (String plan ) {
82+ PlanStatus existingStatus = asyncPlanningStates .get (plan );
83+ if (existingStatus == null ) {
84+ throw new NoSuchPlanIdException ("Cannot find plan with id %s" , plan );
85+ }
86+
87+ return asyncPlanningStates .get (plan );
88+ }
89+
90+ void markAsyncPlanAsComplete (String plan ) {
91+ PlanStatus existingStatus = asyncPlanningStates .get (plan );
92+ Preconditions .checkArgument (existingStatus != null , "Cannot find plan %s" , plan );
93+ Preconditions .checkArgument (
94+ existingStatus == PlanStatus .SUBMITTED ,
95+ "Cannot mark plan %s as completed as it is %s" ,
96+ plan ,
97+ existingStatus );
98+ asyncPlanningStates .put (plan , PlanStatus .COMPLETED );
99+ }
100+
101+ void markAsyncPlanFailed (String plan ) {
102+ PlanStatus existingStatus = asyncPlanningStates .get (plan );
103+ Preconditions .checkArgument (existingStatus != null , "Cannot find plan %s" , plan );
104+ Preconditions .checkArgument (
105+ existingStatus == PlanStatus .SUBMITTED ,
106+ "Cannot mark plan %s as completed as it is %s" ,
107+ plan ,
108+ existingStatus );
109+ asyncPlanningStates .put (plan , PlanStatus .FAILED );
110+ }
111+
71112 List <FileScanTask > fileScanTasksForPlanTask (String planTaskKey ) {
72113 List <FileScanTask > tasks = planTaskToFileScanTasks .get (planTaskKey );
73114 if (tasks == null ) {
@@ -120,13 +161,27 @@ Pair<List<FileScanTask>, String> initialScanTasksFor(String planId) {
120161 return Pair .of (initialEntry .getValue (), initialEntry .getKey ());
121162 }
122163
123- void removePlan (String planId ) {
164+ void cancelPlan (String planId ) {
124165 planTaskToNext .entrySet ().removeIf (entry -> entry .getKey ().contains (planId ));
125166 planTaskToFileScanTasks .entrySet ().removeIf (entry -> entry .getKey ().contains (planId ));
167+ // Clear the ongoing plan status in case the planID is an async one.
168+ if (asyncPlanningStates .containsKey (planId )) {
169+ PlanStatus existingStatus = asyncPlanningStates .get (planId );
170+ // No need to fail cancellation if the plan could not be found
171+ if (existingStatus == null ) {
172+ return ;
173+ }
174+
175+ // No need to fail cancellation if the plan already terminated
176+ if (existingStatus == PlanStatus .SUBMITTED ) {
177+ asyncPlanningStates .put (planId , PlanStatus .CANCELLED );
178+ }
179+ }
126180 }
127181
128182 void clear () {
129183 planTaskToFileScanTasks .clear ();
130184 planTaskToNext .clear ();
185+ asyncPlanningStates .clear ();
131186 }
132187}
0 commit comments