flink k8s作业 | 作业弹性横向伸缩存活时间和告警问题

314 阅读6分钟

背景说明

k8s 作业在弹性横向伸缩时,会在AdaptiveScheduler 中重新构建ExecutionGraph,由于作业的uptime指标是在调度器第一次构建ExecutionGraph时,将uptime注册进去的。那么当作业再次进行构建ExecutionGraph时,其uptime当然也会归为0,此时会引起作业的重启告警。

这里有必要说明,作业本身重启和作业弹性横向伸缩引起的重启。作业本身重启,其本质上是会重新创建一个JobMaster,重新创建一个新的调度器,new JobID(),相当于是一次新的作业提交;而作业进行弹性横向伸缩时,从Job这个角度来看,弹性横向伸缩这个操作,它只是增加或者减少了作业的并行度,作业的其他东西都是不怎么改变的,它只是会重现创建ExecutionGraph。

因此,我们在进行作业横向弹性伸缩时,即需要保持住uptime,也需要在告警层面区分开作业自身重启告警和作业弹性横向伸缩重启告警。

解决方案简析

保持住uptime问题

从原理上,我们如果想要在k8s作业进行横向伸缩时,想要保持住作业第一次调度时的注册进ExecutionGraph的uptime指标,那么就需要在第二次构建ExecutionGraph时,记录下第一次的uptime指标。

当然,我们知道构建ExecutionGraph是调度器的事儿,自然而然这种操作也需要在调度器中进行实现。 这里,我们需要清楚2个问题:

  • 1.应该在什么时候记录下来这个uptime
  • 2.在哪里记录下来这个uptime

其实,也就是when 和 where的问题。

如下代码所示,在调度器中AdaptiveScheduler的属性变量中,定义2个变量。

1、一个变量叫firstRunningTimeStamp,即作业第一次的运行时间。 2、第二变量叫firstllSlotsNum,即第一次作业运行申请的slots数量。

public class AdaptiveScheduler
        implements SchedulerNG,
        Created.Context,
        WaitingForResources.Context,
        CreatingExecutionGraph.Context,
        Executing.Context,
        Restarting.Context,
        Failing.Context,
        Finished.Context,
        StopWithSavepoint.Context,
        CheckpointListener {

    private static final Logger LOG = LoggerFactory.getLogger(AdaptiveScheduler.class);

    private final JobGraphJobInformation jobInformation;
    private final VertexParallelismStore initialParallelismStore;

    private final DeclarativeSlotPool declarativeSlotPool;

    private final long initializationTimestamp;

    private final Executor ioExecutor;
    private final ClassLoader userCodeClassLoader;
    private final JobManagerJobMetricGroup jobManagerJobMetricGroup;

    private final CompletedCheckpointStore completedCheckpointStore;
    private final CheckpointIDCounter checkpointIdCounter;
    private final CheckpointsCleaner checkpointsCleaner;

    private final CompletableFuture<JobStatus> jobTerminationFuture = new CompletableFuture<>();

    private final RestartBackoffTimeStrategy restartBackoffTimeStrategy;

    private final ComponentMainThreadExecutor componentMainThreadExecutor;
    private final FatalErrorHandler fatalErrorHandler;

    private final JobStatusListener jobStatusListener;

    private final SlotAllocator slotAllocator;

    private final ScaleUpController scaleUpController;

    private final Duration initialResourceAllocationTimeout;

    private final Duration resourceStabilizationTimeout;

    private final ExecutionGraphFactory executionGraphFactory;

    private State state = new Created(this, LOG);

    private boolean isTransitioningState = false;

    private int numRestarts = 0;
    // 第一次运行的时间
    private long firstRunningTimeStamp = 0L;
    // 第一次申请的slots数量
    private int firstAllSoltsNum = 0;
    
    private final MutableVertexAttemptNumberStore vertexAttemptNumberStore =
            new DefaultVertexAttemptNumberStore();

    private BackgroundTask<ExecutionGraph> backgroundTask = BackgroundTask.finishedBackgroundTask();

    private final SchedulerExecutionMode executionMode;

    private final int parallelismDefault;

    private boolean isCanScaleUp = false;

    private final BoundedFIFOQueue<RootExceptionHistoryEntry> exceptionHistory;

我们在作业第一次进行调度构建ExecutionGraph前,判断这个firstRunningTimeStamp是否为0,如果为0,我们将其初始化。也就记录下来了第一次调度作业申请到的slots资源。

假设,现在作业正在进行横向伸缩,此时,是会重现构建ExecutionGraph的,在构建之后,会在方法org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler#tryToAssignSlots中申请资源,此时,我们边可以做一个判断: 1、如果firstRunningTimeStamp为0,这将executionGraph中属于RUNNING状态的时间记录下来,即初始化firstRunningTimeStamp。 2、如果此时获取到的slots资源和第一次的不一样,说明此时是在进行弹性横向伸缩,此时将上一步已经初始化的firstRunningTimeStamp设置到新的构建的executiinGraph中。 如下代码所示:

@Override
public CreatingExecutionGraph.AssignmentResult tryToAssignSlots(
        CreatingExecutionGraph.ExecutionGraphWithVertexParallelism
                executionGraphWithVertexParallelism) {
    final ExecutionGraph executionGraph =
            executionGraphWithVertexParallelism.getExecutionGraph();

    executionGraph.start(componentMainThreadExecutor);
    executionGraph.transitionToRunning();
    if (firstRunningTimeStamp == 0L) {
        firstRunningTimeStamp = executionGraph.getStatusTimestamp(JobStatus.RUNNING);
        LOG.info("Initializing first ExecutionGraph running time {} successfully for job:{}",
                firstRunningTimeStamp, executionGraph.getJobName());
    }

    LOG.info("Current job:{} firstAllSoltsNum:{} VS nowAllSlotsNum:{},and firstRunningTimeStamp:{}",
            executionGraph.getJobName(),
            firstAllSoltsNum,
            declarativeSlotPool.getAllSlotsInformation().size(),
            firstRunningTimeStamp);
    // 槽位和最初的不相同,说明外部的TM个数发生了变化,可能是伸缩了TM,此时需要延续之前的运行时间
    if (declarativeSlotPool.getAllSlotsInformation().size() != firstAllSoltsNum) {
        executionGraph.setStatusTimestamp(JobStatus.RUNNING, firstRunningTimeStamp);
    }

    executionGraph.setInternalTaskFailuresListener(
            new UpdateSchedulerNgOnInternalFailuresListener(this));

    final VertexParallelism vertexParallelism =
            executionGraphWithVertexParallelism.getVertexParallelism();
    return slotAllocator
            .tryReserveResources(vertexParallelism)
            .map(
                    reservedSlots ->
                            CreatingExecutionGraph.AssignmentResult.success(
                                    assignSlotsToExecutionGraph(executionGraph, reservedSlots)))
            .orElseGet(CreatingExecutionGraph.AssignmentResult::notPossible);
}

以上,便解决了第一个问题:如何在进行横向弹性伸缩时,记录下上一次的uptime问题。

解决作业到达最终状态时uptime上报-1问题

其实,这里还有一个问题:默认的uptime指标类UptimeGauge在获取值的实现中,当作业处于全局终止状态时,即以下3种状态时:

  • CANCELED
  • FINISHED
  • FAILED

都会返回一个-1的时间,这个其实在作业弹性横向伸缩时,作业重新调度期间,是有问题的,此时上报的uptime指标中,是有-1的,反应到监控图上就是折线有断崖。 对于这种问题,为了避免影响其他的逻辑,单独创建一个弹性伸缩时的uptime指标类ScaleUpTimeGauge,如下:

@Override
    public Long getValue() {
        final JobStatus status = jobStatusProvider.getState();

        if (status == JobStatus.RUNNING) {
            // running right now - report the uptime
            final long runningTimestamp = jobStatusProvider.getStatusTimestamp(JobStatus.RUNNING);
            // we use 'Math.max' here to avoid negative timestamps when clocks change
            return Math.max(System.currentTimeMillis() - runningTimestamp, 0);
        } else if (status.isTerminalState()) {
            // 对于作业弹性伸缩时走到CANCELED状态时,应该依旧返回运行时间
            if (status == JobStatus.CANCELED) {
                return jobStatusProvider.getStatusTimestamp(JobStatus.RUNNING);
            }
            // not running any more -> finished or not on leader
            return NO_LONGER_RUNNING;
        } else {
            // not yet running or not up at the moment
            return 0L;
        }
    }
}

为什么要这样呢?这是因为作业在重启期间,作业状态会经历以下几个过程:

  • RUNNING --> CANCELING
  • CANCELING --> CANCELED 这里会到达一个最终状态
  • CANCELED --> RUNNING

当然,改造还没有结束。

还有一个问题:在我们的实现中,uptime、restartingTime、downTime、initialtime等4个指标,是在DefaultExecutionGraphBuilder 类去调用方法buildGraph中去添加的,那么有个问题,就是在横向弹性伸缩去重建ExecutionGraph的时候,如果不加特殊处理,那么注册进去的还是原生的UpTimeGauge类的实现,依旧会有全局状态CANCELED,导致上报-1问题。

该如何解决呢?

在调度器AdaptiveScheduler中,我们有定义一个变量,isCanScaleUp,用来标记当前是否可以弹性伸缩。那么我们就可以根据这个变量在创建ExecutionGraph的时候,根据是否是弹性伸缩时创建的,来注册不同的uptime指标,如果其为ture,那么我们就可以注册ScaleUpTimeGauge类,否则,注册UptimeGauge。

public static DefaultExecutionGraph buildGraph(
        JobGraph jobGraph,
        Configuration jobManagerConfig,
        ScheduledExecutorService futureExecutor,
        Executor ioExecutor,
        ClassLoader classLoader,
        CompletedCheckpointStore completedCheckpointStore,
        CheckpointsCleaner checkpointsCleaner,
        CheckpointIDCounter checkpointIdCounter,
        Time rpcTimeout,
        MetricGroup metrics,
        BlobWriter blobWriter,
        Logger log,
        ShuffleMaster<?> shuffleMaster,
        JobMasterPartitionTracker partitionTracker,
        TaskDeploymentDescriptorFactory.PartitionLocationConstraint partitionLocationConstraint,
        ExecutionDeploymentListener executionDeploymentListener,
        ExecutionStateUpdateListener executionStateUpdateListener,
        long initializationTimestamp,
        VertexAttemptNumberStore vertexAttemptNumberStore,
        VertexParallelismStore vertexParallelismStore,// 调整后的并行度
        boolean isCanScaleUp)
        throws JobExecutionException, JobException {

    checkNotNull(jobGraph, "job graph cannot be null");

    final String jobName = jobGraph.getName();
    final JobID jobId = jobGraph.getJobID();

    final JobInformation jobInformation =
            new JobInformation(
                    jobId,
                    jobName,
                    jobGraph.getSerializedExecutionConfig(),
                    jobGraph.getJobConfiguration(),
                    jobGraph.getUserJarBlobKeys(),
                    jobGraph.getClasspaths());

    final int maxPriorAttemptsHistoryLength =
            jobManagerConfig.getInteger(JobManagerOptions.MAX_ATTEMPTS_HISTORY_SIZE);

    final PartitionReleaseStrategy.Factory partitionReleaseStrategyFactory =
            PartitionReleaseStrategyFactoryLoader.loadPartitionReleaseStrategyFactory(
                    jobManagerConfig);

    jobInformation.getJobConfiguration().addAll(jobManagerConfig);
    // create a new execution graph, if none exists so far
    final DefaultExecutionGraph executionGraph;
    try {
        executionGraph =
                new DefaultExecutionGraph(
                        jobInformation,
                        futureExecutor,
                        ioExecutor,
                        rpcTimeout,
                        maxPriorAttemptsHistoryLength,
                        classLoader,
                        blobWriter,
                        partitionReleaseStrategyFactory,
                        shuffleMaster,
                        partitionTracker,
                        partitionLocationConstraint,
                        executionDeploymentListener,
                        executionStateUpdateListener,
                        initializationTimestamp,
                        vertexAttemptNumberStore,
                        vertexParallelismStore);
    } catch (IOException e) {
        throw new JobException("Could not create the ExecutionGraph.", e);
    }

    // set the basic properties

    try {
        executionGraph.setJsonPlan(JsonPlanGenerator.generatePlan(jobGraph));
    } catch (Throwable t) {
        log.warn("Cannot create JSON plan for job", t);
        // give the graph an empty plan
        executionGraph.setJsonPlan("{}");
    }

    // initialize the vertices that have a master initialization hook
    // file output formats create directories here, input formats create splits

    final long initMasterStart = System.nanoTime();
    log.info("Running initialization on master for job {} ({}).", jobName, jobId);

    for (JobVertex vertex : jobGraph.getVertices()) {
        String executableClass = vertex.getInvokableClassName();
        if (executableClass == null || executableClass.isEmpty()) {
            throw new JobSubmissionException(
                    jobId,
                    "The vertex "
                            + vertex.getID()
                            + " ("
                            + vertex.getName()
                            + ") has no invokable class.");
        }

        try {
            vertex.initializeOnMaster(classLoader);
        } catch (Throwable t) {
            throw new JobExecutionException(
                    jobId,
                    "Cannot initialize task '" + vertex.getName() + "': " + t.getMessage(),
                    t);
        }
    }

    log.info(
            "Successfully ran initialization on master in {} ms.",
            (System.nanoTime() - initMasterStart) / 1_000_000);

    // topologically sort the job vertices and attach the graph to the existing one
    List<JobVertex> sortedTopology = jobGraph.getVerticesSortedTopologicallyFromSources();
    if (log.isDebugEnabled()) {
        log.debug(
                "Adding {} vertices from job graph {} ({}).",
                sortedTopology.size(),
                jobName,
                jobId);
    }
    executionGraph.attachJobGraph(sortedTopology);

    if (log.isDebugEnabled()) {
        log.debug(
                "Successfully created execution graph from job graph {} ({}).", jobName, jobId);
    }

    // configure the state checkpointing
    if (isCheckpointingEnabled(jobGraph)) {
        JobCheckpointingSettings snapshotSettings = jobGraph.getCheckpointingSettings();

        // Maximum number of remembered checkpoints
        int historySize = jobManagerConfig.getInteger(WebOptions.CHECKPOINTS_HISTORY_SIZE);

        CheckpointStatsTracker checkpointStatsTracker =
                new CheckpointStatsTracker(
                        historySize,
                        snapshotSettings.getCheckpointCoordinatorConfiguration(),
                        metrics);

        // load the state backend from the application settings
        final StateBackend applicationConfiguredBackend;
        final SerializedValue<StateBackend> serializedAppConfigured =
                snapshotSettings.getDefaultStateBackend();

        if (serializedAppConfigured == null) {
            applicationConfiguredBackend = null;
        } else {
            try {
                applicationConfiguredBackend =
                        serializedAppConfigured.deserializeValue(classLoader);
            } catch (IOException | ClassNotFoundException e) {
                throw new JobExecutionException(
                        jobId, "Could not deserialize application-defined state backend.", e);
            }
        }

        final StateBackend rootBackend;
        try {
            rootBackend =
                    StateBackendLoader.fromApplicationOrConfigOrDefault(
                            applicationConfiguredBackend, jobManagerConfig, classLoader, log);
        } catch (IllegalConfigurationException | IOException | DynamicCodeLoadingException e) {
            throw new JobExecutionException(
                    jobId, "Could not instantiate configured state backend", e);
        }

        // load the checkpoint storage from the application settings
        final CheckpointStorage applicationConfiguredStorage;
        final SerializedValue<CheckpointStorage> serializedAppConfiguredStorage =
                snapshotSettings.getDefaultCheckpointStorage();

        if (serializedAppConfiguredStorage == null) {
            applicationConfiguredStorage = null;
        } else {
            try {
                applicationConfiguredStorage =
                        serializedAppConfiguredStorage.deserializeValue(classLoader);
            } catch (IOException | ClassNotFoundException e) {
                throw new JobExecutionException(
                        jobId,
                        "Could not deserialize application-defined checkpoint storage.",
                        e);
            }
        }

        final CheckpointStorage rootStorage;
        try {
            rootStorage =
                    CheckpointStorageLoader.load(
                            applicationConfiguredStorage,
                            null,
                            rootBackend,
                            jobManagerConfig,
                            classLoader,
                            log);
        } catch (IllegalConfigurationException | DynamicCodeLoadingException e) {
            throw new JobExecutionException(
                    jobId, "Could not instantiate configured checkpoint storage", e);
        }

        // instantiate the user-defined checkpoint hooks

        final SerializedValue<MasterTriggerRestoreHook.Factory[]> serializedHooks =
                snapshotSettings.getMasterHooks();
        final List<MasterTriggerRestoreHook<?>> hooks;

        if (serializedHooks == null) {
            hooks = Collections.emptyList();
        } else {
            final MasterTriggerRestoreHook.Factory[] hookFactories;
            try {
                hookFactories = serializedHooks.deserializeValue(classLoader);
            } catch (IOException | ClassNotFoundException e) {
                throw new JobExecutionException(
                        jobId, "Could not instantiate user-defined checkpoint hooks", e);
            }

            final Thread thread = Thread.currentThread();
            final ClassLoader originalClassLoader = thread.getContextClassLoader();
            thread.setContextClassLoader(classLoader);

            try {
                hooks = new ArrayList<>(hookFactories.length);
                for (MasterTriggerRestoreHook.Factory factory : hookFactories) {
                    hooks.add(MasterHooks.wrapHook(factory.create(), classLoader));
                }
            } finally {
                thread.setContextClassLoader(originalClassLoader);
            }
        }

        final CheckpointCoordinatorConfiguration chkConfig =
                snapshotSettings.getCheckpointCoordinatorConfiguration();

        executionGraph.enableCheckpointing(
                chkConfig,
                hooks,
                checkpointIdCounter,
                completedCheckpointStore,
                rootBackend,
                rootStorage,
                checkpointStatsTracker,
                checkpointsCleaner);
    }

    // create all the metrics for the Execution Graph
    if (isCanScaleUp) {
        metrics.removeMetric(ScaleUpTimeGauge.METRIC_NAME);
        metrics.gauge(ScaleUpTimeGauge.METRIC_NAME, new ScaleUpTimeGauge(executionGraph));
    } else {
        metrics.removeMetric(UpTimeGauge.METRIC_NAME);
        metrics.gauge(UpTimeGauge.METRIC_NAME, new UpTimeGauge(executionGraph));
    }
    metrics.removeMetric(RestartTimeGauge.METRIC_NAME);
    metrics.gauge(RestartTimeGauge.METRIC_NAME, new RestartTimeGauge(executionGraph));
    metrics.removeMetric(DownTimeGauge.METRIC_NAME);
    metrics.gauge(DownTimeGauge.METRIC_NAME, new DownTimeGauge(executionGraph));
    metrics.removeMetric(InitialTimeGauge.METRIC_NAME);
    metrics.gauge(InitialTimeGauge.METRIC_NAME, new InitialTimeGauge(executionGraph));

    return executionGraph;
}

如此,我们便解决了上面提到的问题。