@PublicEvolving public class JobManagerOptions extends Object
Modifier and Type | Class and Description |
---|---|
static class |
JobManagerOptions.HybridPartitionDataConsumeConstraint
Constraints of upstream hybrid partition data consumption by downstream.
|
static class |
JobManagerOptions.JobStoreType
Type of job store implementation.
|
static class |
JobManagerOptions.SchedulerType
Type of scheduler implementation.
|
Modifier and Type | Field and Description |
---|---|
static ConfigOption<MemorySize> |
ADAPTIVE_BATCH_SCHEDULER_AVG_DATA_VOLUME_PER_TASK
|
static ConfigOption<Integer> |
ADAPTIVE_BATCH_SCHEDULER_DEFAULT_SOURCE_PARALLELISM
|
static ConfigOption<Integer> |
ADAPTIVE_BATCH_SCHEDULER_MAX_PARALLELISM
Deprecated.
|
static ConfigOption<Integer> |
ADAPTIVE_BATCH_SCHEDULER_MIN_PARALLELISM
Deprecated.
|
static ConfigOption<String> |
ADDRESS
The config parameter defining the network address to connect to for communication with the
job manager.
|
static ConfigOption<String> |
ARCHIVE_DIR
The location where the JobManager stores the archives of completed jobs.
|
static ConfigOption<String> |
BIND_HOST
The local address of the network interface that the job manager binds to.
|
static ConfigOption<Duration> |
BLOCK_SLOW_NODE_DURATION
Deprecated.
|
static ConfigOption<String> |
EXECUTION_FAILOVER_STRATEGY
This option specifies the failover strategy, i.e. how the job computation recovers from task
failures.
|
static ConfigOption<String> |
FAILURE_ENRICHERS_LIST
Flag indicating whether JobManager should load available Failure Enricher plugins at startup.
|
static ConfigOption<JobManagerOptions.HybridPartitionDataConsumeConstraint> |
HYBRID_PARTITION_DATA_CONSUME_CONSTRAINT |
static ConfigOption<Integer> |
JOB_MANAGER_FUTURE_POOL_SIZE |
static ConfigOption<MemorySize> |
JOB_MANAGER_HEAP_MEMORY
Deprecated.
use
TOTAL_FLINK_MEMORY for standalone setups and TOTAL_PROCESS_MEMORY for containerized setups. |
static ConfigOption<Integer> |
JOB_MANAGER_HEAP_MEMORY_MB
Deprecated.
use
TOTAL_FLINK_MEMORY for standalone setups and TOTAL_PROCESS_MEMORY for containerized setups. |
static ConfigOption<Integer> |
JOB_MANAGER_IO_POOL_SIZE |
static ConfigOption<String> |
JOB_MANAGER_RESOURCE_ID
The JobManager's ResourceID.
|
static ConfigOption<Long> |
JOB_STORE_CACHE_SIZE
The job store cache size in bytes which is used to keep completed jobs in memory.
|
static ConfigOption<Long> |
JOB_STORE_EXPIRATION_TIME
The time in seconds after which a completed job expires and is purged from the job store.
|
static ConfigOption<Integer> |
JOB_STORE_MAX_CAPACITY
The max number of completed jobs that can be kept in the job store.
|
static ConfigOption<JobManagerOptions.JobStoreType> |
JOB_STORE_TYPE
Config parameter determining the job store implementation in session cluster.
|
static ConfigOption<Boolean> |
JVM_DIRECT_MEMORY_LIMIT_ENABLED
Off-heap Memory size for the JobManager.
|
static ConfigOption<MemorySize> |
JVM_HEAP_MEMORY
JVM Heap Memory size for the JobManager.
|
static ConfigOption<MemorySize> |
JVM_METASPACE
JVM Metaspace Size for the JobManager.
|
static ConfigOption<Float> |
JVM_OVERHEAD_FRACTION
Fraction of Total Process Memory to be reserved for JVM Overhead.
|
static ConfigOption<MemorySize> |
JVM_OVERHEAD_MAX
Max JVM Overhead size for the TaskExecutors.
|
static ConfigOption<MemorySize> |
JVM_OVERHEAD_MIN
Min JVM Overhead size for the JobManager.
|
static ConfigOption<Integer> |
MAX_ATTEMPTS_HISTORY_SIZE
The maximum number of historical execution attempts kept in history.
|
static MemorySize |
MIN_JVM_HEAP_SIZE |
static ConfigOption<Integer> |
MIN_PARALLELISM_INCREASE |
static ConfigOption<MemorySize> |
OFF_HEAP_MEMORY
Off-heap Memory size for the JobManager.
|
static ConfigOption<Boolean> |
PARTITION_RELEASE_DURING_JOB_EXECUTION
Config parameter controlling whether partitions should already be released during the job
execution.
|
static ConfigOption<Integer> |
PORT
The config parameter defining the network port to connect to for communication with the job
manager.
|
static ConfigOption<Duration> |
RESOURCE_STABILIZATION_TIMEOUT |
static ConfigOption<Duration> |
RESOURCE_WAIT_TIMEOUT |
static ConfigOption<Boolean> |
RETRIEVE_TASK_MANAGER_HOSTNAME
Flag indicating whether JobManager would retrieve canonical host name of TaskManager during
registration.
|
static ConfigOption<Integer> |
RPC_BIND_PORT
The local port that the job manager binds to.
|
static ConfigOption<JobManagerOptions.SchedulerType> |
SCHEDULER
Config parameter determining the scheduler implementation.
|
static ConfigOption<SchedulerExecutionMode> |
SCHEDULER_MODE |
static ConfigOption<Long> |
SLOT_IDLE_TIMEOUT
The timeout in milliseconds for a idle slot in Slot Pool.
|
static ConfigOption<Long> |
SLOT_REQUEST_TIMEOUT
The timeout in milliseconds for requesting a slot from Slot Pool.
|
static ConfigOption<Boolean> |
SPECULATIVE_ENABLED
Deprecated.
|
static ConfigOption<Integer> |
SPECULATIVE_MAX_CONCURRENT_EXECUTIONS
Deprecated.
|
static ConfigOption<MemorySize> |
TOTAL_FLINK_MEMORY
Total Flink Memory size for the JobManager.
|
static ConfigOption<MemorySize> |
TOTAL_PROCESS_MEMORY
Total Process Memory size for the JobManager.
|
public static final MemorySize MIN_JVM_HEAP_SIZE
public static final ConfigOption<String> ADDRESS
This value is only interpreted in setups where a single JobManager with static name or address exists (simple standalone setups, or container setups with dynamic service name resolution). It is not used in many high-availability setups, when a leader-election service (like ZooKeeper) is used to elect and discover the JobManager leader from potentially multiple standby JobManagers.
public static final ConfigOption<String> BIND_HOST
public static final ConfigOption<Integer> PORT
Like ADDRESS
, this value is only interpreted in setups where a
single JobManager with static name/address and port exists (simple standalone setups, or
container setups with dynamic service name resolution). This config option is not used in
many high-availability setups, when a leader-election service (like ZooKeeper) is used to
elect and discover the JobManager leader from potentially multiple standby JobManagers.
public static final ConfigOption<Integer> RPC_BIND_PORT
@Deprecated public static final ConfigOption<MemorySize> JOB_MANAGER_HEAP_MEMORY
TOTAL_FLINK_MEMORY
for standalone setups and TOTAL_PROCESS_MEMORY
for containerized setups.@Deprecated public static final ConfigOption<Integer> JOB_MANAGER_HEAP_MEMORY_MB
TOTAL_FLINK_MEMORY
for standalone setups and TOTAL_PROCESS_MEMORY
for containerized setups.public static final ConfigOption<MemorySize> TOTAL_PROCESS_MEMORY
public static final ConfigOption<MemorySize> TOTAL_FLINK_MEMORY
public static final ConfigOption<MemorySize> JVM_HEAP_MEMORY
public static final ConfigOption<MemorySize> OFF_HEAP_MEMORY
public static final ConfigOption<Boolean> JVM_DIRECT_MEMORY_LIMIT_ENABLED
public static final ConfigOption<MemorySize> JVM_METASPACE
public static final ConfigOption<MemorySize> JVM_OVERHEAD_MIN
public static final ConfigOption<MemorySize> JVM_OVERHEAD_MAX
public static final ConfigOption<Float> JVM_OVERHEAD_FRACTION
public static final ConfigOption<Integer> MAX_ATTEMPTS_HISTORY_SIZE
public static final ConfigOption<String> FAILURE_ENRICHERS_LIST
Example:
jobmanager.failure-enrichers = org.apache.flink.test.plugin.jar.failure.TypeFailureEnricher, org.apache.flink.runtime.failure.FailureEnricherUtilsTest$TestEnricher
public static final ConfigOption<String> EXECUTION_FAILOVER_STRATEGY
public static final ConfigOption<String> ARCHIVE_DIR
public static final ConfigOption<Long> JOB_STORE_CACHE_SIZE
public static final ConfigOption<Long> JOB_STORE_EXPIRATION_TIME
public static final ConfigOption<Integer> JOB_STORE_MAX_CAPACITY
public static final ConfigOption<JobManagerOptions.JobStoreType> JOB_STORE_TYPE
public static final ConfigOption<Boolean> RETRIEVE_TASK_MANAGER_HOSTNAME
public static final ConfigOption<Integer> JOB_MANAGER_FUTURE_POOL_SIZE
public static final ConfigOption<Integer> JOB_MANAGER_IO_POOL_SIZE
public static final ConfigOption<Long> SLOT_REQUEST_TIMEOUT
public static final ConfigOption<Long> SLOT_IDLE_TIMEOUT
public static final ConfigOption<JobManagerOptions.SchedulerType> SCHEDULER
public static final ConfigOption<SchedulerExecutionMode> SCHEDULER_MODE
public static final ConfigOption<Integer> MIN_PARALLELISM_INCREASE
public static final ConfigOption<Duration> RESOURCE_WAIT_TIMEOUT
public static final ConfigOption<Duration> RESOURCE_STABILIZATION_TIMEOUT
public static final ConfigOption<Boolean> PARTITION_RELEASE_DURING_JOB_EXECUTION
@Deprecated public static final ConfigOption<Integer> ADAPTIVE_BATCH_SCHEDULER_MIN_PARALLELISM
BatchExecutionOptions.ADAPTIVE_AUTO_PARALLELISM_MIN_PARALLELISM
.@Deprecated public static final ConfigOption<Integer> ADAPTIVE_BATCH_SCHEDULER_MAX_PARALLELISM
BatchExecutionOptions.ADAPTIVE_AUTO_PARALLELISM_MAX_PARALLELISM
.@Deprecated public static final ConfigOption<MemorySize> ADAPTIVE_BATCH_SCHEDULER_AVG_DATA_VOLUME_PER_TASK
@Deprecated public static final ConfigOption<Integer> ADAPTIVE_BATCH_SCHEDULER_DEFAULT_SOURCE_PARALLELISM
@Deprecated public static final ConfigOption<Boolean> SPECULATIVE_ENABLED
BatchExecutionOptions.SPECULATIVE_ENABLED
.@Deprecated public static final ConfigOption<Integer> SPECULATIVE_MAX_CONCURRENT_EXECUTIONS
BatchExecutionOptions.SPECULATIVE_MAX_CONCURRENT_EXECUTIONS
.@Deprecated public static final ConfigOption<Duration> BLOCK_SLOW_NODE_DURATION
BatchExecutionOptions.BLOCK_SLOW_NODE_DURATION
.public static final ConfigOption<String> JOB_MANAGER_RESOURCE_ID
public static final ConfigOption<JobManagerOptions.HybridPartitionDataConsumeConstraint> HYBRID_PARTITION_DATA_CONSUME_CONSTRAINT
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.