Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-33354][runtime] Cache TaskInformation and JobInformation to avoid deserializing duplicate big objects #23599

Merged
merged 4 commits into from
Nov 7, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@
import org.apache.flink.runtime.jobgraph.DistributionPattern;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.shuffle.ShuffleDescriptor;
import org.apache.flink.runtime.taskexecutor.ShuffleDescriptorsCache;
import org.apache.flink.runtime.util.GroupCache;
import org.apache.flink.util.CompressedSerializedValue;
import org.apache.flink.util.Preconditions;

Expand Down Expand Up @@ -167,7 +167,7 @@ public ShuffleDescriptor[] getShuffleDescriptors() {
public void tryLoadAndDeserializeShuffleDescriptors(
@Nullable PermanentBlobService blobService,
JobID jobId,
ShuffleDescriptorsCache shuffleDescriptorsCache)
GroupCache<JobID, PermanentBlobKey, ShuffleDescriptorGroup> shuffleDescriptorsCache)
throws IOException {
if (inputChannels != null) {
return;
Expand All @@ -190,13 +190,14 @@ private void tryLoadAndDeserializeShuffleDescriptorGroup(
@Nullable PermanentBlobService blobService,
JobID jobId,
MaybeOffloaded<ShuffleDescriptorGroup> serializedShuffleDescriptors,
ShuffleDescriptorsCache shuffleDescriptorsCache)
GroupCache<JobID, PermanentBlobKey, ShuffleDescriptorGroup> shuffleDescriptorsCache)
throws IOException, ClassNotFoundException {
if (serializedShuffleDescriptors instanceof Offloaded) {
PermanentBlobKey blobKey =
((Offloaded<ShuffleDescriptorGroup>) serializedShuffleDescriptors)
.serializedValueKey;
ShuffleDescriptorGroup shuffleDescriptorGroup = shuffleDescriptorsCache.get(blobKey);
ShuffleDescriptorGroup shuffleDescriptorGroup =
shuffleDescriptorsCache.get(jobId, blobKey);
if (shuffleDescriptorGroup == null) {
Preconditions.checkNotNull(blobService);
// NOTE: Do not delete the ShuffleDescriptor BLOBs since it may be needed again
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,19 +23,22 @@
import org.apache.flink.runtime.blob.PermanentBlobService;
import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore;
import org.apache.flink.runtime.clusterframework.types.AllocationID;
import org.apache.flink.runtime.deployment.TaskDeploymentDescriptorFactory.ShuffleDescriptorGroup;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.executiongraph.JobInformation;
import org.apache.flink.runtime.executiongraph.TaskInformation;
import org.apache.flink.runtime.taskexecutor.ShuffleDescriptorsCache;
import org.apache.flink.util.FileUtils;
import org.apache.flink.runtime.util.GroupCache;
import org.apache.flink.util.InstantiationUtil;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.SerializedValue;

import javax.annotation.Nullable;

import java.io.BufferedInputStream;
import java.io.File;
import java.io.IOException;
import java.io.Serializable;
import java.nio.file.Files;
import java.util.List;

/**
Expand Down Expand Up @@ -97,11 +100,23 @@ public Offloaded(PermanentBlobKey serializedValueKey) {
}
}

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

/** Serialized task information or <tt>null</tt> if offloaded. */
private MaybeOffloaded<TaskInformation> serializedTaskInformation;
/** Serialized task information if non-offloaded or <tt>PermanentBlobKey</tt> if offloaded. */
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,39 +166,43 @@ public TaskDeploymentDescriptor(
}

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

/**
* Return the sub task's serialized task 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 SerializedValue<TaskInformation> getSerializedTaskInformation() {
public TaskInformation getTaskInformation() throws IOException, ClassNotFoundException {
if (taskInformation != null) {
return taskInformation;
}
if (serializedTaskInformation instanceof NonOffloaded) {
NonOffloaded<TaskInformation> taskInformation =
(NonOffloaded<TaskInformation>) serializedTaskInformation;
return taskInformation.serializedValue;
} else {
throw new IllegalStateException(
"Trying to work with offloaded serialized job information.");
return taskInformation.serializedValue.deserializeValue(getClass().getClassLoader());
}
throw new IllegalStateException(
"Trying to work with offloaded serialized task information.");
Comment on lines 199 to +205
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about

        Preconditions.checkState(
                serializedJobInformation instanceof NonOffloaded,
                 "Trying to work with offloaded serialized job information.");
        NonOffloaded<JobInformation> jobInformation =
                (NonOffloaded<JobInformation>) serializedJobInformation;
        return jobInformation.serializedValue.deserializeValue(getClass().getClassLoader());


?

}

/**
Expand Down Expand Up @@ -242,7 +261,9 @@ public AllocationID getAllocationId() {
*/
public void loadBigData(
@Nullable PermanentBlobService blobService,
ShuffleDescriptorsCache shuffleDescriptorsCache)
GroupCache<JobID, PermanentBlobKey, JobInformation> jobInformationCache,
GroupCache<JobID, PermanentBlobKey, TaskInformation> taskInformationCache,
GroupCache<JobID, PermanentBlobKey, ShuffleDescriptorGroup> shuffleDescriptorsCache)
throws IOException, ClassNotFoundException {

// re-integrate offloaded job info from blob
Expand All @@ -253,13 +274,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(
new BufferedInputStream(Files.newInputStream(dataFile.toPath())),
getClass().getClassLoader());
jobInformationCache.put(jobId, jobInfoKey, jobInformation);
}
this.jobInformation = jobInformation.deepCopy();
huwh marked this conversation as resolved.
Show resolved Hide resolved
}

// re-integrate offloaded task info from blob
Expand All @@ -269,23 +296,25 @@ 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(
new BufferedInputStream(Files.newInputStream(dataFile.toPath())),
getClass().getClassLoader());
taskInformationCache.put(jobId, taskInfoKey, taskInformation);
}
this.taskInformation = taskInformation.deepCopy();
}

for (InputGateDeploymentDescriptor inputGate : inputGates) {
inputGate.tryLoadAndDeserializeShuffleDescriptors(
blobService, jobId, shuffleDescriptorsCache);
}

// make sure that the serialized job and task information fields are filled
Preconditions.checkNotNull(serializedJobInformation);
Preconditions.checkNotNull(serializedTaskInformation);
}

@Override
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,43 @@ public String getInvokableClassName() {
public Configuration getTaskConfiguration() {
return taskConfiguration;
}

public TaskInformation deepCopy() {
return new TaskInformation(
getJobVertexId(),
getTaskName(),
getNumberOfSubtasks(),
getMaxNumberOfSubtasks(),
getInvokableClassName(),
// Return the new Configuration to avoid shared conf being changed.
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);
}
}
Loading