blob: a7690c962eaca40701a27337cd84cbfdd48efeb5 [file] [log] [blame]
/*
* 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.samza.runtime;
import java.time.Duration;
import org.apache.samza.annotation.InterfaceStability;
import org.apache.samza.context.ExternalContext;
import org.apache.samza.job.ApplicationStatus;
/**
* The primary means of managing execution of the {@link org.apache.samza.application.SamzaApplication} at runtime.
*
* <p>
* Implementation Notes: implementation of {@link ApplicationRunner} must have a public default constructor
* #ApplicationRunner(SamzaApplication, Config)
*/
@InterfaceStability.Evolving
public interface ApplicationRunner {
/**
* This is like {@link #run(ExternalContext)}, except it provides a null {@link ExternalContext}.
*/
default void run() {
run(null);
}
/**
* Deploy and run the Samza jobs to execute {@link org.apache.samza.application.SamzaApplication}.
* It is non-blocking so it doesn't wait for the application running.
* @param externalContext nullable {@link ExternalContext} to pass through to the application
*/
void run(ExternalContext externalContext);
/**
* Kill the Samza jobs represented by {@link org.apache.samza.application.SamzaApplication}
* It is non-blocking so it doesn't wait for the application stopping.
*/
void kill();
/**
* Get the collective status of the Samza jobs represented by {@link org.apache.samza.application.SamzaApplication}.
* Returns {@link ApplicationStatus} object.
*
* @return the current status of an instance of {@link org.apache.samza.application.SamzaApplication}
*/
ApplicationStatus status();
/**
* Waits until the application finishes.
*/
void waitForFinish();
/**
* Waits for {@code timeout} duration for the application to finish.
*
* @param timeout time to wait for the application to finish
* @return true - application finished before timeout
* false - otherwise
*/
boolean waitForFinish(Duration timeout);
}