/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package org.apache.flink.streaming.runtime.tasks; import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.streaming.api.checkpoint.ExternallyInducedSource; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.util.FlinkException; /** * {@link StreamTask} for executing a {@link StreamSource}. * * <p>One important aspect of this is that the checkpointing and the emission of elements must never * occur at the same time. The execution must be serial. This is achieved by having the contract * with the StreamFunction that it must only modify its state or emit elements in * a synchronized block that locks on the lock Object. Also, the modification of the state * and the emission of elements must happen in the same block of code that is protected by the * synchronized block. * * @param <OUT> Type of the output elements of this source. * @param <SRC> Type of the source function for the stream source operator * @param <OP> Type of the stream source operator */ @Internal public class SourceStreamTask<OUT, SRC extends SourceFunction<OUT>, OP extends StreamSource<OUT, SRC>> extends StreamTask<OUT, OP> { private volatile boolean externallyInducedCheckpoints; @Override protected void init() { // does not hold any resources, so no initialization needed // we check if the source is actually inducing the checkpoints, rather // than the trigger ch SourceFunction<?> source = headOperator.getUserFunction(); if (source instanceof ExternallyInducedSource) { externallyInducedCheckpoints = true; ExternallyInducedSource.CheckpointTrigger triggerHook = new ExternallyInducedSource.CheckpointTrigger() { @Override public void triggerCheckpoint(long checkpointId) throws FlinkException { // TODO - we need to see how to derive those. We should probably not encode this in the // TODO - source's trigger message, but do a handshake in this task between the trigger // TODO - message from the master, and the source's trigger notification final CheckpointOptions checkpointOptions = CheckpointOptions.forFullCheckpoint(); final long timestamp = System.currentTimeMillis(); final CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, timestamp); try { SourceStreamTask.super.triggerCheckpoint(checkpointMetaData, checkpointOptions); } catch (RuntimeException | FlinkException e) { throw e; } catch (Exception e) { throw new FlinkException(e.getMessage(), e); } } }; ((ExternallyInducedSource<?, ?>) source).setCheckpointTrigger(triggerHook); } } @Override protected void cleanup() { // does not hold any resources, so no cleanup needed } @Override protected void run() throws Exception { headOperator.run(getCheckpointLock(), getStreamStatusMaintainer()); } @Override protected void cancelTask() throws Exception { if (headOperator != null) { headOperator.cancel(); } } // ------------------------------------------------------------------------ // Checkpointing // ------------------------------------------------------------------------ @Override public boolean triggerCheckpoint(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions) throws Exception { if (!externallyInducedCheckpoints) { return super.triggerCheckpoint(checkpointMetaData, checkpointOptions); } else { // we do not trigger checkpoints here, we simply state whether we can trigger them synchronized (getCheckpointLock()) { return isRunning(); } } } }