Skip to content

Commit

Permalink
[FLINK-33354][runtime] Cache TaskInformation and JobInformation to av…
Browse files Browse the repository at this point in the history
…oid deserializing duplicate big objects
  • Loading branch information
1996fanrui committed Oct 26, 2023
1 parent f76dfc0 commit dfcfdbe
Show file tree
Hide file tree
Showing 10 changed files with 327 additions and 80 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.flink.runtime.executiongraph.TaskInformation;
import org.apache.flink.runtime.util.GroupCache;
import org.apache.flink.util.FileUtils;
import org.apache.flink.util.InstantiationUtil;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.SerializedValue;

Expand Down Expand Up @@ -99,10 +100,22 @@ public Offloaded(PermanentBlobKey serializedValueKey) {
}

/** Serialized job information if non-offloaded or <tt>PermanentBlobKey</tt> if offloaded. */
private MaybeOffloaded<JobInformation> serializedJobInformation;
private final MaybeOffloaded<JobInformation> serializedJobInformation;

/** Serialized task information if non-offloaded or <tt>PermanentBlobKey</tt> if offloaded. */
private MaybeOffloaded<TaskInformation> serializedTaskInformation;
private final MaybeOffloaded<TaskInformation> serializedTaskInformation;

/**
* The job information, it isn't null when serializedJobInformation is offloaded and after
* {@link #loadBigData}.
*/
private transient JobInformation jobInformation;

/**
* The task information, it isn't null when serializedTaskInformation is offloaded and after
* {@link #loadBigData}.
*/
private transient TaskInformation taskInformation;

/**
* The ID referencing the job this task belongs to.
Expand Down Expand Up @@ -151,13 +164,6 @@ public TaskDeploymentDescriptor(
this.inputGates = Preconditions.checkNotNull(inputGateDeploymentDescriptors);
}

/**
* Return the sub task's serialized job information.
*
* @return serialized job information (may throw {@link IllegalStateException} if {@link
* #loadBigData} is not called beforehand).
* @throws IllegalStateException If job information is offloaded to BLOB store.
*/
public SerializedValue<JobInformation> getSerializedJobInformation() {
if (serializedJobInformation instanceof NonOffloaded) {
NonOffloaded<JobInformation> jobInformation =
Expand All @@ -170,12 +176,45 @@ public SerializedValue<JobInformation> getSerializedJobInformation() {
}

/**
* Return the sub task's serialized task information.
* Return the sub task's job information.
*
* @return job information (may throw {@link IllegalStateException} if {@link #loadBigData} is
* not called beforehand).
* @throws IllegalStateException If job information is offloaded to BLOB store.
*/
public JobInformation getJobInformation() throws IOException, ClassNotFoundException {
if (jobInformation != null) {
return jobInformation;
}
if (serializedJobInformation instanceof NonOffloaded) {
NonOffloaded<JobInformation> jobInformation =
(NonOffloaded<JobInformation>) serializedJobInformation;
return jobInformation.serializedValue.deserializeValue(getClass().getClassLoader());
}
throw new IllegalStateException(
"Trying to work with offloaded serialized job information.");
}

/**
* Return the sub task's task information.
*
* @return serialized task information (may throw {@link IllegalStateException} if {@link
* #loadBigData} is not called beforehand)).
* @return task information (may throw {@link IllegalStateException} if {@link #loadBigData} is
* not called beforehand)).
* @throws IllegalStateException If job information is offloaded to BLOB store.
*/
public TaskInformation getTaskInformation() throws IOException, ClassNotFoundException {
if (taskInformation != null) {
return taskInformation;
}
if (serializedTaskInformation instanceof NonOffloaded) {
NonOffloaded<TaskInformation> taskInformation =
(NonOffloaded<TaskInformation>) serializedTaskInformation;
return taskInformation.serializedValue.deserializeValue(getClass().getClassLoader());
}
throw new IllegalStateException(
"Trying to work with offloaded serialized task information.");
}

public SerializedValue<TaskInformation> getSerializedTaskInformation() {
if (serializedTaskInformation instanceof NonOffloaded) {
NonOffloaded<TaskInformation> taskInformation =
Expand Down Expand Up @@ -243,6 +282,8 @@ public AllocationID getAllocationId() {
*/
public void loadBigData(
@Nullable PermanentBlobService blobService,
GroupCache<JobID, PermanentBlobKey, JobInformation> jobInformationCache,
GroupCache<JobID, PermanentBlobKey, TaskInformation> taskInformationCache,
GroupCache<JobID, PermanentBlobKey, ShuffleDescriptorGroup> shuffleDescriptorsCache)
throws IOException, ClassNotFoundException {

Expand All @@ -254,13 +295,19 @@ public void loadBigData(

Preconditions.checkNotNull(blobService);

final File dataFile = blobService.getFile(jobId, jobInfoKey);
// NOTE: Do not delete the job info BLOB since it may be needed again during recovery.
// (it is deleted automatically on the BLOB server and cache when the job
// enters a terminal state)
SerializedValue<JobInformation> serializedValue =
SerializedValue.fromBytes(FileUtils.readAllBytes(dataFile.toPath()));
serializedJobInformation = new NonOffloaded<>(serializedValue);
JobInformation jobInformation = jobInformationCache.get(jobId, jobInfoKey);
if (jobInformation == null) {
final File dataFile = blobService.getFile(jobId, jobInfoKey);
// NOTE: Do not delete the job info BLOB since it may be needed again during
// recovery. (it is deleted automatically on the BLOB server and cache when the job
// enters a terminal state)
jobInformation =
InstantiationUtil.deserializeObject(
FileUtils.readAllBytes(dataFile.toPath()),
getClass().getClassLoader());
jobInformationCache.put(jobId, jobInfoKey, jobInformation);
}
this.jobInformation = jobInformation.deepCopy();
}

// re-integrate offloaded task info from blob
Expand All @@ -270,13 +317,19 @@ public void loadBigData(

Preconditions.checkNotNull(blobService);

final File dataFile = blobService.getFile(jobId, taskInfoKey);
// NOTE: Do not delete the task info BLOB since it may be needed again during recovery.
// (it is deleted automatically on the BLOB server and cache when the job
// enters a terminal state)
SerializedValue<TaskInformation> serializedValue =
SerializedValue.fromBytes(FileUtils.readAllBytes(dataFile.toPath()));
serializedTaskInformation = new NonOffloaded<>(serializedValue);
TaskInformation taskInformation = taskInformationCache.get(jobId, taskInfoKey);
if (taskInformation == null) {
final File dataFile = blobService.getFile(jobId, taskInfoKey);
// NOTE: Do not delete the task info BLOB since it may be needed again during
// recovery. (it is deleted automatically on the BLOB server and cache when the job
// enters a terminal state)
taskInformation =
InstantiationUtil.deserializeObject(
FileUtils.readAllBytes(dataFile.toPath()),
getClass().getClassLoader());
taskInformationCache.put(jobId, taskInfoKey, taskInformation);
}
this.taskInformation = taskInformation.deepCopy();
}

for (InputGateDeploymentDescriptor inputGate : inputGates) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,13 +21,18 @@
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.UnmodifiableConfiguration;
import org.apache.flink.runtime.blob.PermanentBlobKey;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.SerializedValue;

import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableCollection;
import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableList;

import java.io.Serializable;
import java.net.URL;
import java.util.Collection;
import java.util.Objects;

/** Container class for job information which is stored in the {@link ExecutionGraph}. */
public class JobInformation implements Serializable {
Expand All @@ -44,13 +49,13 @@ public class JobInformation implements Serializable {
private final SerializedValue<ExecutionConfig> serializedExecutionConfig;

/** Configuration of the job. */
private final Configuration jobConfiguration;
private final UnmodifiableConfiguration jobConfiguration;

/** Blob keys for the required jar files. */
private final Collection<PermanentBlobKey> requiredJarFileBlobKeys;
private final ImmutableCollection<PermanentBlobKey> requiredJarFileBlobKeys;

/** URLs specifying the classpath to add to the class loader. */
private final Collection<URL> requiredClasspathURLs;
private final ImmutableCollection<URL> requiredClasspathURLs;

public JobInformation(
JobID jobId,
Expand All @@ -62,9 +67,12 @@ public JobInformation(
this.jobId = Preconditions.checkNotNull(jobId);
this.jobName = Preconditions.checkNotNull(jobName);
this.serializedExecutionConfig = Preconditions.checkNotNull(serializedExecutionConfig);
this.jobConfiguration = Preconditions.checkNotNull(jobConfiguration);
this.requiredJarFileBlobKeys = Preconditions.checkNotNull(requiredJarFileBlobKeys);
this.requiredClasspathURLs = Preconditions.checkNotNull(requiredClasspathURLs);
this.jobConfiguration =
new UnmodifiableConfiguration(Preconditions.checkNotNull(jobConfiguration));
this.requiredJarFileBlobKeys =
ImmutableList.copyOf(Preconditions.checkNotNull(requiredJarFileBlobKeys));
this.requiredClasspathURLs =
ImmutableList.copyOf(Preconditions.checkNotNull(requiredClasspathURLs));
}

public JobID getJobId() {
Expand All @@ -79,18 +87,51 @@ public SerializedValue<ExecutionConfig> getSerializedExecutionConfig() {
return serializedExecutionConfig;
}

public Configuration getJobConfiguration() {
public UnmodifiableConfiguration getJobConfiguration() {
return jobConfiguration;
}

public Collection<PermanentBlobKey> getRequiredJarFileBlobKeys() {
public ImmutableCollection<PermanentBlobKey> getRequiredJarFileBlobKeys() {
return requiredJarFileBlobKeys;
}

public Collection<URL> getRequiredClasspathURLs() {
public ImmutableCollection<URL> getRequiredClasspathURLs() {
return requiredClasspathURLs;
}

// All fields are immutable, so return this directly.
public JobInformation deepCopy() {
return this;
}

@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
JobInformation that = (JobInformation) o;
return Objects.equals(jobId, that.jobId)
&& Objects.equals(jobName, that.jobName)
&& Objects.equals(serializedExecutionConfig, that.serializedExecutionConfig)
&& Objects.equals(jobConfiguration, that.jobConfiguration)
&& Objects.equals(requiredJarFileBlobKeys, that.requiredJarFileBlobKeys)
&& Objects.equals(requiredClasspathURLs, that.requiredClasspathURLs);
}

@Override
public int hashCode() {
return Objects.hash(
jobId,
jobName,
serializedExecutionConfig,
jobConfiguration,
requiredJarFileBlobKeys,
requiredClasspathURLs);
}

// ------------------------------------------------------------------------

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.flink.util.Preconditions;

import java.io.Serializable;
import java.util.Objects;

/**
* Container class for operator/task specific information which are stored at the {@link
Expand Down Expand Up @@ -88,4 +89,42 @@ public String getInvokableClassName() {
public Configuration getTaskConfiguration() {
return taskConfiguration;
}

public TaskInformation deepCopy() {
return new TaskInformation(
getJobVertexId(),
getTaskName(),
getNumberOfSubtasks(),
getMaxNumberOfSubtasks(),
getInvokableClassName(),
new Configuration(getTaskConfiguration()));
}

@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
TaskInformation that = (TaskInformation) o;
return numberOfSubtasks == that.numberOfSubtasks
&& maxNumberOfSubtasks == that.maxNumberOfSubtasks
&& Objects.equals(jobVertexId, that.jobVertexId)
&& Objects.equals(taskName, that.taskName)
&& Objects.equals(invokableClassName, that.invokableClassName)
&& Objects.equals(taskConfiguration, that.taskConfiguration);
}

@Override
public int hashCode() {
return Objects.hash(
jobVertexId,
taskName,
numberOfSubtasks,
maxNumberOfSubtasks,
invokableClassName,
taskConfiguration);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -300,6 +300,8 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway {

private final ThreadInfoSampleService threadInfoSampleService;

private final GroupCache<JobID, PermanentBlobKey, JobInformation> jobInformationCache;
private final GroupCache<JobID, PermanentBlobKey, TaskInformation> taskInformationCache;
private final GroupCache<JobID, PermanentBlobKey, ShuffleDescriptorGroup>
shuffleDescriptorsCache;

Expand Down Expand Up @@ -378,6 +380,8 @@ public TaskExecutor(
taskExecutorServices.getSlotAllocationSnapshotPersistenceService();

this.sharedResources = taskExecutorServices.getSharedResources();
this.jobInformationCache = taskExecutorServices.getJobInformationCache();
this.taskInformationCache = taskExecutorServices.getTaskInformationCache();
this.shuffleDescriptorsCache = taskExecutorServices.getShuffleDescriptorCache();
}

Expand Down Expand Up @@ -508,6 +512,8 @@ public CompletableFuture<Void> onStop() {
changelogStoragesManager.shutdown();
channelStateExecutorFactoryManager.shutdown();

jobInformationCache.clear();
taskInformationCache.clear();
shuffleDescriptorsCache.clear();

Preconditions.checkState(jobTable.isEmpty());
Expand Down Expand Up @@ -672,7 +678,10 @@ public CompletableFuture<Acknowledge> submitTask(
// re-integrate offloaded data and deserialize shuffle descriptors
try {
tdd.loadBigData(
taskExecutorBlobService.getPermanentBlobService(), shuffleDescriptorsCache);
taskExecutorBlobService.getPermanentBlobService(),
jobInformationCache,
taskInformationCache,
shuffleDescriptorsCache);
} catch (IOException | ClassNotFoundException e) {
throw new TaskSubmissionException(
"Could not re-integrate offloaded TaskDeploymentDescriptor data.", e);
Expand All @@ -682,12 +691,8 @@ public CompletableFuture<Acknowledge> submitTask(
final JobInformation jobInformation;
final TaskInformation taskInformation;
try {
jobInformation =
tdd.getSerializedJobInformation()
.deserializeValue(getClass().getClassLoader());
taskInformation =
tdd.getSerializedTaskInformation()
.deserializeValue(getClass().getClassLoader());
jobInformation = tdd.getJobInformation();
taskInformation = tdd.getTaskInformation();
} catch (IOException | ClassNotFoundException e) {
throw new TaskSubmissionException(
"Could not deserialize the job or task information.", e);
Expand Down Expand Up @@ -1907,6 +1912,8 @@ private void releaseJobResources(JobID jobId, Exception cause) {
changelogStoragesManager.releaseResourcesForJob(jobId);
currentSlotOfferPerJob.remove(jobId);
channelStateExecutorFactoryManager.releaseResourcesForJob(jobId);
jobInformationCache.clearCacheForGroup(jobId);
taskInformationCache.clearCacheForGroup(jobId);
shuffleDescriptorsCache.clearCacheForGroup(jobId);
fileMergingManager.releaseMergingSnapshotManagerForJob(jobId);
}
Expand Down
Loading

0 comments on commit dfcfdbe

Please sign in to comment.