Skip to content
Open
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
@@ -0,0 +1,24 @@
<table class="configuration table table-bordered">
<thead>
<tr>
<th class="text-left" style="width: 20%">Key</th>
<th class="text-left" style="width: 15%">Default</th>
<th class="text-left" style="width: 10%">Type</th>
<th class="text-left" style="width: 55%">Description</th>
</tr>
</thead>
<tbody>
<tr>
<td><h5>application-result-store.delete-on-commit</h5></td>
<td style="word-wrap: break-word;">true</td>
<td>Boolean</td>
<td>Determines whether application results should be automatically removed from the underlying application result store when the corresponding entity transitions into a clean state. If false, the cleaned application results are, instead, marked as clean to indicate their state. In this case, Flink no longer has ownership and the resources need to be cleaned up by the user.</td>
</tr>
<tr>
<td><h5>application-result-store.storage-path</h5></td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
<td>Defines where application results should be stored. This should be an underlying file-system that provides read-after-write consistency. By default, this is <code class="highlighter-rouge">{high-availability.storageDir}/application-result-store/{high-availability.cluster-id}</code>.</td>
</tr>
</tbody>
</table>
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
<table class="configuration table table-bordered">
<thead>
<tr>
<th class="text-left" style="width: 20%">Key</th>
<th class="text-left" style="width: 15%">Default</th>
<th class="text-left" style="width: 10%">Type</th>
<th class="text-left" style="width: 55%">Description</th>
</tr>
</thead>
<tbody>
<tr>
<td><h5>application-result-store.delete-on-commit</h5></td>
<td style="word-wrap: break-word;">true</td>
<td>Boolean</td>
<td>Determines whether application results should be automatically removed from the underlying application result store when the corresponding entity transitions into a clean state. If false, the cleaned application results are, instead, marked as clean to indicate their state. In this case, Flink no longer has ownership and the resources need to be cleaned up by the user.</td>
</tr>
<tr>
<td><h5>application-result-store.storage-path</h5></td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
<td>Defines where application results should be stored. This should be an underlying file-system that provides read-after-write consistency. By default, this is <code class="highlighter-rouge">{high-availability.storageDir}/application-result-store/{high-availability.cluster-id}</code>.</td>
</tr>
</tbody>
</table>
Original file line number Diff line number Diff line change
Expand Up @@ -8,12 +8,6 @@
</tr>
</thead>
<tbody>
<tr>
<td><h5>job-result-store.delete-on-commit</h5></td>
<td style="word-wrap: break-word;">true</td>
<td>Boolean</td>
<td>Determines whether job results should be automatically removed from the underlying job result store when the corresponding entity transitions into a clean state. If false, the cleaned job results are, instead, marked as clean to indicate their state. In this case, Flink no longer has ownership and the resources need to be cleaned up by the user.</td>
</tr>
<tr>
<td><h5>job-result-store.storage-path</h5></td>
<td style="word-wrap: break-word;">(none)</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,12 @@
<td>Boolean</td>
<td>Defines whether a suspended ZooKeeper connection will be treated as an error that causes the leader information to be invalidated or not. In case you set this option to <code class="highlighter-rouge">true</code>, Flink will wait until a ZooKeeper connection is marked as lost before it revokes the leadership of components. This has the effect that Flink is more resilient against temporary connection instabilities at the cost of running more likely into timing issues with ZooKeeper.</td>
</tr>
<tr>
<td><h5>high-availability.zookeeper.path.applications</h5></td>
<td style="word-wrap: break-word;">"/applications"</td>
<td>String</td>
<td>ZooKeeper root path (ZNode) for applications</td>
</tr>
<tr>
<td><h5>high-availability.zookeeper.path.execution-plans</h5></td>
<td style="word-wrap: break-word;">"/execution-plans"</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,12 @@
<td>Boolean</td>
<td>Defines whether a suspended ZooKeeper connection will be treated as an error that causes the leader information to be invalidated or not. In case you set this option to <code class="highlighter-rouge">true</code>, Flink will wait until a ZooKeeper connection is marked as lost before it revokes the leadership of components. This has the effect that Flink is more resilient against temporary connection instabilities at the cost of running more likely into timing issues with ZooKeeper.</td>
</tr>
<tr>
<td><h5>high-availability.zookeeper.path.applications</h5></td>
<td style="word-wrap: break-word;">"/applications"</td>
<td>String</td>
<td>ZooKeeper root path (ZNode) for applications</td>
</tr>
<tr>
<td><h5>high-availability.zookeeper.path.execution-plans</h5></td>
<td style="word-wrap: break-word;">"/execution-plans"</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,12 +8,6 @@
</tr>
</thead>
<tbody>
<tr>
<td><h5>job-result-store.delete-on-commit</h5></td>
<td style="word-wrap: break-word;">true</td>
<td>Boolean</td>
<td>Determines whether job results should be automatically removed from the underlying job result store when the corresponding entity transitions into a clean state. If false, the cleaned job results are, instead, marked as clean to indicate their state. In this case, Flink no longer has ownership and the resources need to be cleaned up by the user.</td>
</tr>
<tr>
<td><h5>job-result-store.storage-path</h5></td>
<td style="word-wrap: break-word;">(none)</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,8 @@ public static final class Sections {
"common_high_availability_zk";
public static final String COMMON_HIGH_AVAILABILITY_JOB_RESULT_STORE =
"common_high_availability_jrs";
public static final String COMMON_HIGH_AVAILABILITY_APPLICATION_RESULT_STORE =
"common_high_availability_ars";
public static final String COMMON_MEMORY = "common_memory";
public static final String COMMON_MISCELLANEOUS = "common_miscellaneous";

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,8 @@ org.apache.flink.cep.pattern.conditions.IterativeCondition.filter(java.lang.Obje
org.apache.flink.cep.pattern.conditions.SimpleCondition.filter(java.lang.Object, org.apache.flink.cep.pattern.conditions.IterativeCondition$Context): Argument leaf type org.apache.flink.cep.pattern.conditions.IterativeCondition$Context does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.client.program.StreamContextEnvironment.execute(org.apache.flink.streaming.api.graph.StreamGraph): Argument leaf type org.apache.flink.streaming.api.graph.StreamGraph does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.client.program.StreamContextEnvironment.executeAsync(org.apache.flink.streaming.api.graph.StreamGraph): Argument leaf type org.apache.flink.streaming.api.graph.StreamGraph does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.client.program.StreamContextEnvironment.setAsContext(org.apache.flink.core.execution.PipelineExecutorServiceLoader, org.apache.flink.configuration.Configuration, java.lang.ClassLoader, boolean, boolean, org.apache.flink.api.common.ApplicationID, java.util.Collection): Argument leaf type org.apache.flink.core.execution.PipelineExecutorServiceLoader does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.client.program.StreamContextEnvironment.setAsContext(org.apache.flink.core.execution.PipelineExecutorServiceLoader, org.apache.flink.configuration.Configuration, java.lang.ClassLoader, boolean, boolean, org.apache.flink.api.common.ApplicationID, org.apache.flink.client.program.JarInfo, java.util.Collection): Argument leaf type org.apache.flink.client.program.JarInfo does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.client.program.StreamContextEnvironment.setAsContext(org.apache.flink.core.execution.PipelineExecutorServiceLoader, org.apache.flink.configuration.Configuration, java.lang.ClassLoader, boolean, boolean, org.apache.flink.api.common.ApplicationID, org.apache.flink.client.program.JarInfo, java.util.Collection): Argument leaf type org.apache.flink.core.execution.PipelineExecutorServiceLoader does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.client.program.StreamPlanEnvironment.executeAsync(org.apache.flink.streaming.api.graph.StreamGraph): Argument leaf type org.apache.flink.streaming.api.graph.StreamGraph does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.client.program.StreamPlanEnvironment.getPipeline(): Returned leaf type org.apache.flink.api.dag.Pipeline does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.configuration.ClusterOptions.getSchedulerType(org.apache.flink.configuration.Configuration): Returned leaf type org.apache.flink.configuration.JobManagerOptions$SchedulerType does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3,10 +3,10 @@ Constructor <org.apache.flink.streaming.runtime.io.StreamTaskExternallyInducedSo
Method <org.apache.flink.cep.nfa.sharedbuffer.LockableTypeSerializerSnapshot.getNestedSerializers(org.apache.flink.cep.nfa.sharedbuffer.Lockable$LockableTypeSerializer)> calls method <org.apache.flink.cep.nfa.sharedbuffer.Lockable$LockableTypeSerializer.getElementSerializer()> in (LockableTypeSerializerSnapshot.java:58)
Method <org.apache.flink.runtime.blob.BlobInputStream.read()> calls method <org.apache.flink.runtime.blob.BlobKey.getHash()> in (BlobInputStream.java:127)
Method <org.apache.flink.runtime.blob.BlobInputStream.read([B, int, int)> calls method <org.apache.flink.runtime.blob.BlobKey.getHash()> in (BlobInputStream.java:163)
Method <org.apache.flink.runtime.blob.BlobOutputStream.receiveAndCheckPutResponse(java.io.InputStream, java.security.MessageDigest, org.apache.flink.runtime.blob.BlobKey$BlobType)> calls method <org.apache.flink.runtime.blob.BlobKey.getHash()> in (BlobOutputStream.java:155)
Method <org.apache.flink.runtime.blob.BlobUtils.checkAndDeleteCorruptedBlobs(java.nio.file.Path, org.slf4j.Logger)> calls method <org.apache.flink.runtime.blob.BlobKey.getHash()> in (BlobUtils.java:514)
Method <org.apache.flink.runtime.blob.FileSystemBlobStore.get(java.lang.String, java.io.File, org.apache.flink.runtime.blob.BlobKey)> calls method <org.apache.flink.runtime.blob.BlobKey.getHash()> in (FileSystemBlobStore.java:129)
Method <org.apache.flink.runtime.executiongraph.Execution.finishPartitionsAndUpdateConsumers()> calls method <org.apache.flink.runtime.executiongraph.ExecutionVertex.finishPartitionsIfNeeded()> in (Execution.java:1031)
Method <org.apache.flink.runtime.blob.BlobOutputStream.receiveAndCheckPutResponse(java.io.InputStream, java.security.MessageDigest, org.apache.flink.runtime.blob.BlobKey$BlobType)> calls method <org.apache.flink.runtime.blob.BlobKey.getHash()> in (BlobOutputStream.java:186)
Method <org.apache.flink.runtime.blob.BlobUtils.checkAndDeleteCorruptedBlobs(java.nio.file.Path, org.slf4j.Logger)> calls method <org.apache.flink.runtime.blob.BlobKey.getHash()> in (BlobUtils.java:665)
Method <org.apache.flink.runtime.blob.FileSystemBlobStore.get(java.lang.String, java.io.File, org.apache.flink.runtime.blob.BlobKey)> calls method <org.apache.flink.runtime.blob.BlobKey.getHash()> in (FileSystemBlobStore.java:148)
Method <org.apache.flink.runtime.executiongraph.Execution.finishPartitionsAndUpdateConsumers()> calls method <org.apache.flink.runtime.executiongraph.ExecutionVertex.finishPartitionsIfNeeded()> in (Execution.java:1202)
Method <org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServicesWithLeadershipControl.grantDispatcherLeadership()> calls method <org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedLeaderService.grantLeadership()> in (EmbeddedHaServicesWithLeadershipControl.java:83)
Method <org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServicesWithLeadershipControl.grantJobMasterLeadership(org.apache.flink.api.common.JobID)> calls method <org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedLeaderService.grantLeadership()> in (EmbeddedHaServicesWithLeadershipControl.java:95)
Method <org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServicesWithLeadershipControl.grantResourceManagerLeadership()> calls method <org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedLeaderService.grantLeadership()> in (EmbeddedHaServicesWithLeadershipControl.java:109)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.flink.api.common.JobInfo;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.client.cli.ClientOptions;
import org.apache.flink.client.program.JarInfo;
import org.apache.flink.client.program.PackagedProgram;
import org.apache.flink.client.program.ProgramInvocationException;
import org.apache.flink.client.program.StreamContextEnvironment;
Expand Down Expand Up @@ -92,6 +93,7 @@ public static void executeProgram(
enforceSingleJobExecution,
suppressSysout,
null,
null,
Collections.emptyList());
}

Expand All @@ -102,6 +104,7 @@ public static void executeProgram(
boolean enforceSingleJobExecution,
boolean suppressSysout,
@Nullable ApplicationID applicationId,
@Nullable JarInfo userJarInfo,
Collection<JobInfo> allRecoveredJobInfos)
throws ProgramInvocationException {
checkNotNull(executorServiceLoader);
Expand All @@ -121,6 +124,7 @@ public static void executeProgram(
enforceSingleJobExecution,
suppressSysout,
applicationId,
userJarInfo,
allRecoveredJobInfos);

// For DataStream v2.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,15 +26,19 @@
import org.apache.flink.configuration.ApplicationOptionsInternal;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.DeploymentOptions;
import org.apache.flink.runtime.application.AbstractApplication;
import org.apache.flink.runtime.dispatcher.ApplicationBootstrap;
import org.apache.flink.runtime.dispatcher.Dispatcher;
import org.apache.flink.runtime.dispatcher.DispatcherFactory;
import org.apache.flink.runtime.dispatcher.DispatcherId;
import org.apache.flink.runtime.dispatcher.PartialDispatcherServices;
import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobPersistenceComponents;
import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithPersistenceComponents;
import org.apache.flink.runtime.dispatcher.runner.AbstractDispatcherLeaderProcess;
import org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherGatewayService;
import org.apache.flink.runtime.highavailability.ApplicationResult;
import org.apache.flink.runtime.highavailability.ApplicationResultStore;
import org.apache.flink.runtime.highavailability.JobResultStore;
import org.apache.flink.runtime.jobmanager.ApplicationStore;
import org.apache.flink.runtime.jobmanager.ExecutionPlanWriter;
import org.apache.flink.runtime.jobmaster.JobResult;
import org.apache.flink.runtime.rpc.RpcService;
Expand Down Expand Up @@ -90,8 +94,12 @@ public AbstractDispatcherLeaderProcess.DispatcherGatewayService create(
DispatcherId fencingToken,
Collection<ExecutionPlan> recoveredJobs,
Collection<JobResult> recoveredDirtyJobResults,
Collection<AbstractApplication> recoveredApplications,
Collection<ApplicationResult> recoveredDirtyApplicationResults,
ExecutionPlanWriter executionPlanWriter,
JobResultStore jobResultStore) {
JobResultStore jobResultStore,
ApplicationStore applicationStore,
ApplicationResultStore applicationResultStore) {

final List<JobInfo> recoveredJobInfos = getRecoveredJobInfos(recoveredJobs);
final List<JobInfo> recoveredTerminalJobInfos =
Expand Down Expand Up @@ -126,12 +134,16 @@ public AbstractDispatcherLeaderProcess.DispatcherGatewayService create(
fencingToken,
recoveredJobs,
recoveredDirtyJobResults,
recoveredApplications,
recoveredDirtyApplicationResults,
(dispatcherGateway, scheduledExecutor, errorHandler) ->
new ApplicationBootstrap(bootstrapApplication),
PartialDispatcherServicesWithJobPersistenceComponents.from(
PartialDispatcherServicesWithPersistenceComponents.from(
partialDispatcherServices,
executionPlanWriter,
jobResultStore));
jobResultStore,
applicationStore,
applicationResultStore));
} catch (Exception e) {
throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e);
}
Expand Down
Loading