@@ -118,6 +118,12 @@ public Offloaded(PermanentBlobKey serializedValueKey) {
118
118
*/
119
119
private transient TaskInformation taskInformation ;
120
120
121
+ /** Information to restore the task. This can be null if there is no state to restore. */
122
+ @ Nullable private final MaybeOffloaded <JobManagerTaskRestore > serializedTaskRestore ;
123
+
124
+ /** Information to restore the task. This can be null if there is no state to restore. */
125
+ @ Nullable private transient JobManagerTaskRestore taskRestore ;
126
+
121
127
/**
122
128
* The ID referencing the job this task belongs to.
123
129
*
@@ -138,16 +144,13 @@ public Offloaded(PermanentBlobKey serializedValueKey) {
138
144
/** The list of consumed intermediate result partitions. */
139
145
private final List <InputGateDeploymentDescriptor > inputGates ;
140
146
141
- /** Information to restore the task. This can be null if there is no state to restore. */
142
- @ Nullable private final JobManagerTaskRestore taskRestore ;
143
-
144
147
public TaskDeploymentDescriptor (
145
148
JobID jobId ,
146
149
MaybeOffloaded <JobInformation > serializedJobInformation ,
147
150
MaybeOffloaded <TaskInformation > serializedTaskInformation ,
148
151
ExecutionAttemptID executionAttemptId ,
149
152
AllocationID allocationId ,
150
- @ Nullable JobManagerTaskRestore taskRestore ,
153
+ @ Nullable MaybeOffloaded < JobManagerTaskRestore > serializedTaskRestore ,
151
154
List <ResultPartitionDeploymentDescriptor > resultPartitionDeploymentDescriptors ,
152
155
List <InputGateDeploymentDescriptor > inputGateDeploymentDescriptors ) {
153
156
@@ -159,7 +162,7 @@ public TaskDeploymentDescriptor(
159
162
this .executionId = Preconditions .checkNotNull (executionAttemptId );
160
163
this .allocationId = Preconditions .checkNotNull (allocationId );
161
164
162
- this .taskRestore = taskRestore ;
165
+ this .serializedTaskRestore = serializedTaskRestore ;
163
166
164
167
this .producedPartitions = Preconditions .checkNotNull (resultPartitionDeploymentDescriptors );
165
168
this .inputGates = Preconditions .checkNotNull (inputGateDeploymentDescriptors );
@@ -241,8 +244,16 @@ public List<InputGateDeploymentDescriptor> getInputGates() {
241
244
}
242
245
243
246
@ Nullable
244
- public JobManagerTaskRestore getTaskRestore () {
245
- return taskRestore ;
247
+ public JobManagerTaskRestore getTaskRestore () throws IOException , ClassNotFoundException {
248
+ if (taskRestore != null || serializedTaskRestore == null ) {
249
+ return taskRestore ;
250
+ }
251
+ if (serializedTaskRestore instanceof NonOffloaded ) {
252
+ NonOffloaded <JobManagerTaskRestore > taskRestore =
253
+ (NonOffloaded <JobManagerTaskRestore >) serializedTaskRestore ;
254
+ return taskRestore .serializedValue .deserializeValue (getClass ().getClassLoader ());
255
+ }
256
+ throw new IllegalStateException ("Trying to work with offloaded serialized task restore." );
246
257
}
247
258
248
259
public AllocationID getAllocationId () {
@@ -311,6 +322,21 @@ public void loadBigData(
311
322
this .taskInformation = taskInformation .deepCopy ();
312
323
}
313
324
325
+ if (serializedTaskRestore == null ) {
326
+ this .taskRestore = null ;
327
+ } else if (serializedTaskRestore instanceof Offloaded ) {
328
+ final PermanentBlobKey blobKey =
329
+ ((Offloaded <JobManagerTaskRestore >) serializedTaskRestore ).serializedValueKey ;
330
+
331
+ Preconditions .checkNotNull (blobService );
332
+
333
+ final File dataFile = blobService .getFile (jobId , blobKey );
334
+ taskRestore =
335
+ InstantiationUtil .deserializeObject (
336
+ new BufferedInputStream (Files .newInputStream (dataFile .toPath ())),
337
+ getClass ().getClassLoader ());
338
+ }
339
+
314
340
for (InputGateDeploymentDescriptor inputGate : inputGates ) {
315
341
inputGate .tryLoadAndDeserializeShuffleDescriptors (
316
342
blobService , jobId , shuffleDescriptorsCache );
0 commit comments